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/07/02 18:00:30 UTC

[01/26] git commit: SLIDER-162: home dir created by hdfs if user lacks permissions (and hdfs can be impersonated)

Repository: incubator-slider
Updated Branches:
  refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it 81a85495d -> 621a981cb (forced update)


SLIDER-162: home dir created by hdfs if user lacks permissions (and hdfs can be impersonated)


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: e3c5d9d8e7198e3a30027a412b9348926900c14a
Parents: 9619c33
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jun 30 16:14:15 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jun 30 16:14:15 2014 +0100

----------------------------------------------------------------------
 .../funtest/framework/AgentUploads.groovy       | 13 ++---
 .../funtest/framework/FileUploader.groovy       | 51 +++++++++++++++++---
 .../lifecycle/AgentCommandTestBase.groovy       | 10 +++-
 3 files changed, 57 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e3c5d9d8/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentUploads.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentUploads.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentUploads.groovy
index 3570183..2cec5c2 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentUploads.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentUploads.groovy
@@ -27,11 +27,9 @@ import org.apache.hadoop.fs.FileSystem as HadoopFS
 @Slf4j
 class AgentUploads implements FuntestProperties {
   final Configuration conf
-  private final FileUploader uploader
-  private final HadoopFS clusterFS
-  private final Path homeDir
-  
-
+  public final FileUploader uploader
+  public final HadoopFS clusterFS
+  public final Path homeDir
 
   AgentUploads(Configuration conf) {
     this.conf = conf
@@ -52,6 +50,8 @@ class AgentUploads implements FuntestProperties {
         homeDir,
         AGENT_TAR_FILENAME)
 
+    //create the home dir or fail
+    uploader.mkHomeDir()
     // Upload the agent tarball
     uploader.copyIfOutOfDate(localAgentTar, agentTarballPath, force)
 
@@ -59,10 +59,7 @@ class AgentUploads implements FuntestProperties {
     // Upload the agent.ini
     def agentIniPath = new Path(homeDir, AGENT_INI)
     uploader.copyIfOutOfDate(localAgentIni, agentIniPath, force)
-    
     return [agentTarballPath, agentIniPath]
-
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e3c5d9d8/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
index 4f61730..2dc85be 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
@@ -20,12 +20,14 @@ package org.apache.slider.funtest.framework
 
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileSystem as HadoopFS
 import org.apache.hadoop.fs.FileUtil
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.fs.permission.FsPermission
 import org.apache.hadoop.security.AccessControlException
 import org.apache.hadoop.security.UserGroupInformation
-import org.apache.hadoop.fs.FileSystem as HadoopFS
+
+@SuppressWarnings("GroovyOctalInteger")
 @Slf4j
 class FileUploader {
   final Configuration conf
@@ -60,18 +62,20 @@ class FileUploader {
     }
     if (toCopy) {
       log.info("Copying $src to $destPath")
+      def dir = destPath.getParent()
       try {
         fs.delete(destPath, true)
-        fs.mkdirs(destPath.getParent(), FsPermission.dirDefault)
+        fs.mkdirs(dir, FsPermission.dirDefault)
         return FileUtil.copy(src, fs, destPath, false, conf)
       } catch (AccessControlException ace) {
-        log.error("No write access to test user home directory. " +
+        log.error("No write access to destination directory $dir" +
                   "Ensure home directory exists and has correct permissions." +
                   ace, ace)
         throw ace
       }
     } else {
-      log.debug("Skipping copy as the destination $destPath considered up to date")
+      log.debug(
+          "Skipping copy as the destination $destPath considered up to date")
       return false;
     }
   }
@@ -79,7 +83,7 @@ class FileUploader {
   public HadoopFS getFileSystem(Path dest) {
     getFileSystem(user, dest)
   }
-  
+
   public HadoopFS getFileSystem() {
     getFileSystem(user, HadoopFS.getDefaultUri(conf))
   }
@@ -88,13 +92,46 @@ class FileUploader {
   public def getFileSystem(
       UserGroupInformation user, final Path path) {
     return getFileSystem(user, path.toUri())
-    
+
   }
+
   public def getFileSystem(
       UserGroupInformation user, final URI uri) {
-    
+
     SudoClosure.sudo(user) {
       HadoopFS.get(uri, conf);
     }
   }
+
+  public def getFileSystemAsUserHdfs() {
+    def hdfs = UserGroupInformation.createRemoteUser("hdfs")
+    getFileSystem(hdfs, HadoopFS.getDefaultUri(conf))
+  }
+
+  /**
+   * Create the home dir. If it can't be created as the user,
+   * try to become the user 'hdfs' and try there, setting the
+   * user and group after.
+   * @return the home dir
+   */
+  public def mkHomeDir() {
+    def fs = fileSystem
+    def home = fs.homeDirectory
+    if (!fs.exists(home)) {
+      try {
+        fs.mkdirs(home)
+      } catch (AccessControlException ace) {
+        log.info("Failed to mkdir $home as $user -impersonating 'hdfs")
+        if (UserGroupInformation.securityEnabled) {
+          // in a secure cluster, we cannot impersonate HDFS, so rethrow
+          throw ace;
+        }
+        //now create as hdfs
+        def FsAsUserHDFS = fileSystemAsUserHdfs
+        FsAsUserHDFS.mkdirs(home, new FsPermission((short) 00755))
+        FsAsUserHDFS.setOwner(home, user.userName, user.primaryGroupName)
+      }
+    }
+    return home
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e3c5d9d8/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
index c3d6bd6..33c0b81 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
@@ -80,6 +80,9 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
   @Before
   public void setupApplicationPackage() {
+    AgentUploads agentUploads = new AgentUploads(SLIDER_CONFIG)
+    agentUploads.uploader.mkHomeDir()
+
     appPkgPath = new Path(clusterFS.homeDirectory, "cmd_log_app_pkg.zip")
     if (!clusterFS.exists(appPkgPath)) {
       clusterFS.delete(appPkgPath, false)
@@ -87,14 +90,17 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
     def pkgPath = folder.newFolder("testpkg")
     File zipFileName = new File(pkgPath, "cmd_log_app_pkg.zip").canonicalFile
-    assume(new File(APP_PKG_DIR).exists(), "App pkg dir not found at $APP_PKG_DIR")
+
+    def localZipDirectory = new File(APP_PKG_DIR)
+    assume(localZipDirectory.exists(), "App pkg dir not found at $APP_PKG_DIR")
 
     zipDir(zipFileName.canonicalPath, APP_PKG_DIR)
 
     // Verify and upload the app pkg
     assume(zipFileName.exists(), "App pkg not found at $zipFileName")
     Path localAppPkg = new Path(zipFileName.toURI());
-    clusterFS.copyFromLocalFile(false, true, localAppPkg, appPkgPath)
+    agentUploads.uploader.copyIfOutOfDate(zipFileName, appPkgPath, false)
+    
   }
 
   public static void logShell(SliderShell shell) {


[24/26] git commit: SLIDER-151 moving rest paths slightly, making more constants, and adding a shared base class AbstractSliderResource for all resources that need at the slider web api

Posted by st...@apache.org.
SLIDER-151 moving rest paths slightly, making more constants, and adding a shared base class AbstractSliderResource for all resources that need at the slider web api


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 5960dd07a2931d397b22f712c2a7ce3c2851a58c
Parents: 6d96ba9
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jun 19 14:54:44 2014 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 17:00:06 2014 +0100

----------------------------------------------------------------------
 .../web/rest/AMWadlGeneratorConfig.java         |  4 +-
 .../web/rest/AbstractSliderResource.java        | 34 ++++++++++++
 .../server/appmaster/web/rest/RestPaths.java    |  3 +-
 .../web/rest/SliderJacksonJaxbJsonProvider.java |  1 -
 .../appmaster/web/rest/agent/AgentResource.java |  6 +--
 .../web/rest/management/ManagementResource.java | 15 +++---
 .../management/resources/ActionsResource.java   | 22 ++++++++
 .../resources/AggregateConfResource.java        |  4 +-
 .../management/resources/ConfTreeResource.java  |  3 +-
 .../web/rest/publisher/PublisherResource.java   | 55 +++++++++++---------
 10 files changed, 103 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java
index 4ae7490..ea9f22b 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java
@@ -22,9 +22,9 @@ import com.sun.jersey.api.wadl.config.WadlGeneratorDescription;
 import java.util.List;
 
 /**
- *
+ * App master's WADL generation support
  */
-public class AMWadlGeneratorConfig extends WadlGeneratorConfig{
+public class AMWadlGeneratorConfig extends WadlGeneratorConfig {
 
   public static final String CLASSNAME = "org.apache.slider.server.appmaster.web.rest.AMWadlGeneratorConfig";
   @Override

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

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
index d55635f..35d0f15 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -26,7 +26,8 @@ public class RestPaths {
   public static final String WS_CONTEXT = "ws";
   public static final String WS_CONTEXT_ROOT = "/" + WS_CONTEXT;
   public static final String SLIDER_CONTEXT_ROOT = WS_CONTEXT_ROOT +"/v1/slider";
-  public static final String SLIDER_SUBPATH_MANAGEMENT = "/mgmt";
+  public static final String MANAGEMENT = "mgmt";
+  public static final String SLIDER_SUBPATH_MANAGEMENT = "/" + MANAGEMENT;
   public static final String SLIDER_SUBPATH_AGENTS = "/agents";
   public static final String SLIDER_SUBPATH_PUBLISHER = "/publisher";
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java
index 95b0aa1..86d68a8 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java
@@ -42,7 +42,6 @@ import javax.ws.rs.ext.Provider;
 public class SliderJacksonJaxbJsonProvider extends JacksonJaxbJsonProvider {
 
   public SliderJacksonJaxbJsonProvider() {
-    super();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
index 96b7b47..d9dcdd5 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
@@ -17,6 +17,7 @@
 package org.apache.slider.server.appmaster.web.rest.agent;
 
 import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
@@ -36,13 +37,12 @@ import javax.ws.rs.core.Response;
  */
 @JsonIgnoreProperties(ignoreUnknown = true)
 @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class AgentResource {
+public class AgentResource extends AbstractSliderResource {
 
-  private final WebAppApi slider;
   private String agent_name;
 
   public AgentResource(WebAppApi slider) {
-    this.slider = slider;
+    super(slider);
   }
 
   private void init(HttpServletResponse res) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
index 94db409..02199cf 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
@@ -18,6 +18,7 @@ package org.apache.slider.server.appmaster.web.rest.management;
 
 import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
 import org.apache.slider.server.appmaster.web.rest.RestPaths;
 import org.apache.slider.server.appmaster.web.rest.management.resources.AggregateConfResource;
 import org.apache.slider.server.appmaster.web.rest.management.resources.ConfTreeResource;
@@ -36,18 +37,18 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriInfo;
+import java.net.URI;
 import java.net.URL;
 
 /**
  *
  */
-public class ManagementResource {
+public class ManagementResource extends AbstractSliderResource {
   protected static final Logger log =
       LoggerFactory.getLogger(ManagementResource.class);
-  private final WebAppApi slider;
 
   public ManagementResource(WebAppApi slider) {
-    this.slider = slider;
+    super(slider);
   }
 
   private void init(HttpServletResponse res) {
@@ -57,7 +58,7 @@ public class ManagementResource {
   @GET
   public Response getWadl (@Context HttpServletRequest request) {
     try {
-      java.net.URI location = new URL(request.getScheme(),
+      URI location = new URL(request.getScheme(),
                                       request.getServerName(),
                                       request.getServerPort(),
                                       "/application.wadl").toURI();
@@ -87,9 +88,9 @@ public class ManagementResource {
     init(res);
     AggregateConfResource aggregateConf =
         ResourceFactory.createAggregateConfResource(getAggregateConf(),
-                                                    uriInfo.getBaseUriBuilder()
-                                                    .path(RestPaths.SLIDER_CONTEXT_ROOT).path(
-                                                    "mgmt/app"));
+      uriInfo.getBaseUriBuilder()
+      .path(RestPaths.SLIDER_CONTEXT_ROOT)
+      .path(RestPaths.MANAGEMENT + "/app"));
     return aggregateConf.getConfTree(config);
   }
 

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

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
index 9df692a..dc26699 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
@@ -38,7 +38,7 @@ public class AggregateConfResource {
   public AggregateConfResource(AggregateConf conf, UriBuilder uriBuilder) {
     if (uriBuilder != null) {
       this.href =
-          uriBuilder.build(null).toASCIIString();
+          uriBuilder.build().toASCIIString();
       resources =
           ResourceFactory.createConfTreeResource(conf.getAppConf(),
                                                  uriBuilder.clone().path(
@@ -63,7 +63,7 @@ public class AggregateConfResource {
   }
 
   private void initConfMap() {
-    confMap = new HashMap<String, ConfTreeResource>();
+    confMap = new HashMap<>();
     confMap.put("internal", internal);
     confMap.put("resources", resources);
     confMap.put("appConf", appConf);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java
index 79f5399..407bab6 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java
@@ -42,8 +42,7 @@ public class ConfTreeResource {
       metadata = confTree.metadata;
       global = confTree.global;
       components = confTree.components;
-      uriBuilder = uriBuilder.clone();
-      this.href = uriBuilder.build(null).toASCIIString();
+      this.href = uriBuilder.build().toASCIIString();
     } else {
       this.href = null;
       this.metadata = null;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5960dd07/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
index a439d9b..89a7d83 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
@@ -26,6 +26,7 @@ import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
 import org.apache.slider.core.registry.docstore.UriMap;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +43,6 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Set;
@@ -52,18 +52,21 @@ import static  org.apache.slider.server.appmaster.web.rest.RestPaths.*;
 /**
  * This publishes configuration sets
  */
-public class PublisherResource {
+public class PublisherResource extends AbstractSliderResource {
   protected static final Logger log =
       LoggerFactory.getLogger(PublisherResource.class);
-  private final WebAppApi slider;
-  public static final String SET_NAME = 
-      "{setname: " + PUBLISHED_CONFIGURATION_SET_REGEXP + "}";
-  private static final String CONFIG =
-      SET_NAME + "/{config: " + PUBLISHED_CONFIGURATION_REGEXP + "}";
+  public static final String SETNAME = "setname";
+  public static final String CLASSPATH = "/classpath";
+  public static final String CONFIG = "config";
+  
+  public static final String SETNAME_PATTERN = 
+      "{"+ SETNAME+": " + PUBLISHED_CONFIGURATION_SET_REGEXP + "}";
+  private static final String CONFIG_PATTERN =
+      SETNAME_PATTERN + "/{"+ CONFIG +": " + PUBLISHED_CONFIGURATION_REGEXP + "}";
   private final StateAccessForProviders appState;
 
   public PublisherResource(WebAppApi slider) {
-    this.slider = slider;
+    super(slider);
     appState = slider.getAppState();
   }
 
@@ -106,7 +109,7 @@ public class PublisherResource {
   }
 
   @GET
-  @Path("/classpath")
+  @Path(CLASSPATH)
   @Produces({MediaType.APPLICATION_JSON})
   public Set<URL> getAMClassPath() {
     URL[] urls = ((URLClassLoader) getClass().getClassLoader()).getURLs();
@@ -114,10 +117,10 @@ public class PublisherResource {
   }
 
   @GET
-  @Path("/"+ SET_NAME)
+  @Path("/"+ SETNAME_PATTERN)
   @Produces({MediaType.APPLICATION_JSON})
   public PublishedConfigSet getPublishedConfiguration(
-      @PathParam("setname") String setname,
+      @PathParam(SETNAME) String setname,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) {
     init(res, uriInfo);
@@ -133,11 +136,11 @@ public class PublisherResource {
   }
 
   @GET
-  @Path("/" + CONFIG)
+  @Path("/" + CONFIG_PATTERN)
   @Produces({MediaType.APPLICATION_JSON})
   public PublishedConfiguration getConfigurationInstance(
-      @PathParam("setname") String setname,
-      @PathParam("config") String config,
+      @PathParam(SETNAME) String setname,
+      @PathParam(CONFIG) String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) {
     init(res, uriInfo);
@@ -164,12 +167,12 @@ public class PublisherResource {
   }
 
   @GET
-  @Path("/" + CONFIG + ".json")
+  @Path("/" + CONFIG_PATTERN + ".json")
   @Produces({MediaType.APPLICATION_JSON})
   public String getConfigurationContentJson(
-      @PathParam("setname") String setname,
+      @PathParam(SETNAME) String setname,
 
-      @PathParam("config") String config,
+      @PathParam(CONFIG) String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) throws IOException {
     return getStringRepresentation(setname, config, uriInfo, res,
@@ -177,11 +180,11 @@ public class PublisherResource {
   }
 
   @GET
-  @Path("/" + CONFIG + ".xml")
+  @Path("/" + CONFIG_PATTERN + ".xml")
   @Produces({MediaType.APPLICATION_XML})
   public String getConfigurationContentXML(
-      @PathParam("setname") String setname,
-      @PathParam("config") String config,
+      @PathParam(SETNAME) String setname,
+      @PathParam(CONFIG) String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) throws IOException {
     return getStringRepresentation(setname, config, uriInfo, res,
@@ -189,12 +192,12 @@ public class PublisherResource {
   }
   
   @GET
-  @Path("/" + CONFIG + ".properties")
+  @Path("/" + CONFIG_PATTERN + ".properties")
   @Produces({MediaType.APPLICATION_XML})
   public String getConfigurationContentProperties(
-      @PathParam("setname") String setname,
+      @PathParam(SETNAME) String setname,
 
-      @PathParam("config") String config,
+      @PathParam(CONFIG) String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) throws IOException {
 
@@ -215,11 +218,11 @@ public class PublisherResource {
   }
 
   @GET
-  @Path("/" + CONFIG +"/{propertyName}")
+  @Path("/" + CONFIG_PATTERN +"/{propertyName}")
   @Produces({MediaType.APPLICATION_JSON})
   public Map<String,String> getConfigurationProperty(
-      @PathParam("setname") String setname,
-      @PathParam("config") String config,
+      @PathParam(SETNAME) String setname,
+      @PathParam(CONFIG) String config,
       @PathParam("propertyName") String propertyName,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) {


[16/26] git commit: Merge branch 'feature/SLIDER-171-slider-on-windows' into develop

Posted by st...@apache.org.
Merge branch 'feature/SLIDER-171-slider-on-windows' into develop


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 0317c411590e378ce8dc03e1b13b7eb1acd0ea32
Parents: 9a54a0b 853a979
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 21:10:16 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 21:10:16 2014 +0100

----------------------------------------------------------------------
 .../slider/client/SliderYarnClientImpl.java     |  3 +-
 .../apache/slider/common/tools/SliderUtils.java |  2 +-
 .../core/launch/ClasspathConstructor.java       |  4 +-
 .../core/launch/JavaCommandLineBuilder.java     |  2 +-
 .../funtest/framework/CommandTestBase.groovy    |  4 +-
 .../funtest/framework/FileUploader.groovy       | 30 ++++--
 .../basic/TestClusterConnectivity.groovy        | 96 ++++++++++++++++++++
 .../lifecycle/TestAgentClusterLifecycle.groovy  |  3 +-
 .../clusters/c6401/slider/slider-client.xml     | 19 ++--
 .../clusters/morzine/slider/log4j.properties    | 83 +++++++++++++++++
 .../clusters/morzine/slider/slider-client.xml   | 76 ++++++++++++++++
 .../clusters/offline/slider/slider-client.xml   |  7 +-
 .../clusters/remote/slider/slider-client.xml    |  5 -
 .../clusters/sandbox/slider/slider-client.xml   |  5 -
 src/test/clusters/ubuntu-secure/operations.md   | 36 ++++----
 15 files changed, 315 insertions(+), 60 deletions(-)
----------------------------------------------------------------------



[20/26] git commit: SLIDER-159 TestLongLivedProcess.testEcho failing

Posted by st...@apache.org.
SLIDER-159 TestLongLivedProcess.testEcho failing


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: e9ddcf0b13beb6aea3822ee0164f6f46f40676f2
Parents: 5bb80a4
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 13:23:54 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 13:23:54 2014 +0100

----------------------------------------------------------------------
 .../services/workflow/ForkedProcessService.java | 22 +-------------
 .../services/workflow/LongLivedProcess.java     | 31 ++++++++++++++++++++
 .../services/workflow/TestLongLivedProcess.java |  5 ++--
 3 files changed, 35 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e9ddcf0b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
index ee68aa4..ccce6cb 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
@@ -279,27 +279,7 @@ public class ForkedProcessService extends AbstractWorkflowExecutorService implem
     if (process == null) {
       return new LinkedList<String>();
     }
-    long start = System.currentTimeMillis();
-    while (System.currentTimeMillis() - start <= duration) {
-      boolean finished;
-      if (finalOutput) {
-        // final flag means block until all data is done
-        finished = process.isFinalOutputProcessed();
-      } else {
-        // there is some output
-        finished = !process.isRecentOutputEmpty();
-      }
-      if (finished) {
-        break;
-      }
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-        break;
-      }
-    }
-    return process.getRecentOutput();
+    return process.getRecentOutput(finalOutput, duration);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e9ddcf0b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
index d9ddecb..05a1c50 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
@@ -357,6 +357,37 @@ public class LongLivedProcess implements Runnable {
   }
 
   /**
+   * Get the recent output from the process, or [] if not defined
+   *
+   * @param finalOutput flag to indicate "wait for the final output of the process"
+   * @param duration the duration, in ms, 
+   * ro wait for recent output to become non-empty
+   * @return a possibly empty list
+   */
+  public List<String> getRecentOutput(boolean finalOutput, int duration) {
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start <= duration) {
+      boolean finishedOutput;
+      if (finalOutput) {
+        // final flag means block until all data is done
+        finishedOutput = isFinalOutputProcessed();
+      } else {
+        // there is some output
+        finishedOutput = !isRecentOutputEmpty();
+      }
+      if (finishedOutput) {
+        break;
+      }
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        break;
+      }
+    }
+    return getRecentOutput();
+  }
+  /**
    * add the recent line to the list of recent lines; deleting
    * an earlier on if the limit is reached.
    *

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e9ddcf0b/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java b/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
index c8a0719..668bcca 100644
--- a/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
+++ b/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
@@ -126,13 +126,14 @@ public class TestLongLivedProcess extends WorkflowServiceTestBase implements
   }
 
   /**
-   * Get the final output. includes a quick sleep for the tail output
+   * Get the final output. 
    * @return the last output
    */
   private List<String> getFinalOutput() {
-    return process.getRecentOutput();
+    return process.getRecentOutput(true, 4000);
   }
 
+
   private LongLivedProcess initProcess(List<String> commands) {
     process = new LongLivedProcess(name.getMethodName(), log, commands);
     process.setLifecycleCallback(this);


[13/26] git commit: SLIDER-190. Add projects to create command logger package as well as Slider app package

Posted by st...@apache.org.
SLIDER-190. Add projects to create command logger package as well as Slider app package


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: e879109a45f255d63e3d32d5f8bb6d40a26e38e9
Parents: f53db87
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Mon Jun 30 19:47:04 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 1 12:56:33 2014 -0700

----------------------------------------------------------------------
 app-packages/command-logger/README.txt          |  22 ++++
 .../command-logger/application-pkg/pom.xml      |  83 ++++++++++++++
 .../src/command_logger/README.txt               |  18 ++++
 .../src/command_logger/operations.log           |  24 +++++
 .../src/packages/tarball/all.xml                |  35 ++++++
 .../command-logger/slider-pkg/appConfig.json    |  27 +++++
 .../slider-pkg/configuration/cl-site.xml        |  34 ++++++
 .../command-logger/slider-pkg/metainfo.xml      |  52 +++++++++
 .../slider-pkg/package/scripts/cl.py            |  89 +++++++++++++++
 .../slider-pkg/package/scripts/params.py        |  31 ++++++
 .../package/templates/operations.log.j2         |  22 ++++
 app-packages/command-logger/slider-pkg/pom.xml  | 107 +++++++++++++++++++
 .../command-logger/slider-pkg/resources.json    |  15 +++
 .../slider-pkg/src/assembly/command-logger.xml  |  78 ++++++++++++++
 app-packages/hbase/appConfig.json               |   4 +-
 pom.xml                                         |   2 +
 16 files changed, 641 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/README.txt b/app-packages/command-logger/README.txt
new file mode 100644
index 0000000..56a8a33
--- /dev/null
+++ b/app-packages/command-logger/README.txt
@@ -0,0 +1,22 @@
+<!---
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+Command logger is a test application used by Slider for functional tests.
+
+Like any typical Slider Application, there is a native application package
+in the form of a tarball that is created by project under application_pkg and
+there is a project under slider_pkg to create the Slider Application Package
+that is consumed by Slider.

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/application-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/pom.xml b/app-packages/command-logger/application-pkg/pom.xml
new file mode 100644
index 0000000..7da7259
--- /dev/null
+++ b/app-packages/command-logger/application-pkg/pom.xml
@@ -0,0 +1,83 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+  <parent>
+    <groupId>org.apache.slider</groupId>
+    <artifactId>slider</artifactId>
+    <version>0.31.0-incubating-SNAPSHOT</version>
+    <relativePath>../../../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>command-logger</artifactId>
+  <packaging>pom</packaging>
+  <name>Command Logger</name>
+  <description>Command Logger</description>
+  <properties>
+    <app.package.name>command-logger</app.package.name>
+  </properties>
+
+  <build>
+    <plugins>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <tarLongFileMode>gnu</tarLongFileMode>
+          <descriptor>src/packages/tarball/all.xml</descriptor>
+          <appendAssemblyId>false</appendAssemblyId>
+          <finalName>${app.package.name}</finalName>
+        </configuration>
+        <executions>
+          <execution>
+            <id>build-tarball</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>3.0</version>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <version>${apache-rat-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>check-licenses</id>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+    <extensions>
+      <extension>
+        <groupId>org.apache.maven.wagon</groupId>
+        <artifactId>wagon-ssh-external</artifactId>
+      </extension>
+    </extensions>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/application-pkg/src/command_logger/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/src/command_logger/README.txt b/app-packages/command-logger/application-pkg/src/command_logger/README.txt
new file mode 100644
index 0000000..19d2e26
--- /dev/null
+++ b/app-packages/command-logger/application-pkg/src/command_logger/README.txt
@@ -0,0 +1,18 @@
+<!---
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+All this project contains is a tarball that has a sample log file.

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/application-pkg/src/command_logger/operations.log
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/src/command_logger/operations.log b/app-packages/command-logger/application-pkg/src/command_logger/operations.log
new file mode 100644
index 0000000..4604cdd
--- /dev/null
+++ b/app-packages/command-logger/application-pkg/src/command_logger/operations.log
@@ -0,0 +1,24 @@
+<!---
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+This is a log for all operations on a Yarn container.
+Container Id: Container001
+Application id: Application001
+---------------
+Time:
+Log:
+---------------

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/application-pkg/src/packages/tarball/all.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/src/packages/tarball/all.xml b/app-packages/command-logger/application-pkg/src/packages/tarball/all.xml
new file mode 100644
index 0000000..5c6080b
--- /dev/null
+++ b/app-packages/command-logger/application-pkg/src/packages/tarball/all.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.1"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.1 http://maven.apache.org/xsd/assembly-1.1.1.xsd">
+  <!--This 'agent-all' id is not appended to the produced bundle because we do this:
+    http://maven.apache.org/plugins/maven-assembly-plugin/faq.html#required-classifiers
+  -->
+  <id>command-logger</id>
+  <formats>
+    <format>tar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>src/command_logger</directory>
+      <outputDirectory>command-logger-app</outputDirectory>
+    </fileSet>
+  </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/appConfig.json b/app-packages/command-logger/slider-pkg/appConfig.json
new file mode 100644
index 0000000..e7f9700
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/appConfig.json
@@ -0,0 +1,27 @@
+{
+    "schema": "http://example.org/specification/v2.0.0",
+    "metadata": {
+    },
+    "global": {
+        "agent.conf": "agent.ini",
+        "application.def": "apache-slider-command-logger.zip",
+        "config_types": "cl-site",
+        "java_home": "/usr/jdk64/jdk1.7.0_45",
+        "package_list": "files/command-logger.tar",
+        "site.global.app_user": "yarn",
+        "site.global.application_id": "CommandLogger",
+        "site.global.app_log_dir": "${AGENT_LOG_ROOT}/app/log",
+        "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
+        "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/command-logger",
+        "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
+        "site.cl-site.logfile.location": "${AGENT_LOG_ROOT}/app/log/operations.log",
+        "site.cl-site.datetime.format": "%A, %d. %B %Y %I:%M%p"
+    },
+    "components": {
+        "COMMAND_LOGGER": {
+        },
+        "slider-appmaster": {
+            "jvm.heapsize": "256M"
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/configuration/cl-site.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/configuration/cl-site.xml b/app-packages/command-logger/slider-pkg/configuration/cl-site.xml
new file mode 100644
index 0000000..977c8bc
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/configuration/cl-site.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+-->
+<configuration>
+  <property>
+    <name>logfile.location</name>
+    <value>/tmp/logfiles/operations.log</value>
+    <description>The location where the log file is stored.</description>
+  </property>
+  <property>
+    <name>datetime.format</name>
+    <value>%A, %d. %B %Y %I:%M%p</value>
+    <description>The format to use.
+    </description>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/metainfo.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/metainfo.xml b/app-packages/command-logger/slider-pkg/metainfo.xml
new file mode 100644
index 0000000..845ad93
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/metainfo.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <application>
+    <name>TEST_COMMAND_LOG</name>
+    <comment>
+      When started it creates a new log file and stores all commands in the
+      log file. When stopped it renames the file.
+    </comment>
+    <version>0.1.0</version>
+    <components>
+      <component>
+        <name>COMMAND_LOGGER</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/cl.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/command_log.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+  </application>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/package/scripts/cl.py
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/package/scripts/cl.py b/app-packages/command-logger/slider-pkg/package/scripts/cl.py
new file mode 100644
index 0000000..6b18faa
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/package/scripts/cl.py
@@ -0,0 +1,89 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+import sys
+import os
+from datetime import datetime
+from resource_management import *
+from resource_management.core.base import Fail
+
+
+class CommandLogger(Script):
+  def install(self, env):
+    self.install_packages(env)
+
+  def configure(self, env):
+    import params
+
+    env.set_params(params)
+
+  def start(self, env):
+    import params
+
+    env.set_params(params)
+    self.configure(env)
+    self.rename_file(env)
+    self.ensure_file(env)
+    self.check_and_log(env, "Starting instance.")
+
+  def stop(self, env):
+    import params
+
+    env.set_params(params)
+    self.check_and_log(env, "Stopping instance.")
+    self.rename_file(env)
+
+  def status(self, env):
+    Logger.info("Returning status as live.")
+
+  def check_and_log(self, env, message):
+    import params
+
+    file_location = params.file_location
+    datetime_format = params.datetime_format
+    if not os.path.isfile(file_location) or not os.access(file_location,
+                                                          os.W_OK):
+      raise Fail("File does not exist or not writable. %s" % file_location)
+    with open(file_location, "a") as logfile:
+      logfile.write("Time: " + datetime.utcnow().strftime(datetime_format) + "\n")
+      logfile.write("Log: " + message + "\n")
+      logfile.write("---------------\n")
+
+  def rename_file(self, env):
+    import params
+
+    file_location = params.file_location
+    if os.path.isfile(file_location) and \
+      os.access(file_location, os.W_OK):
+      new_file_name = \
+        file_location + "." + datetime.utcnow().strftime("%d_%m_%y_%I_%M_%S")
+      os.rename(file_location, new_file_name)
+
+  def ensure_file(self, env):
+    import params
+
+    file_location = params.file_location
+    TemplateConfig( file_location,
+                    owner = params.app_user,
+                    template_tag = None
+    )
+
+if __name__ == "__main__":
+  CommandLogger().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/package/scripts/params.py b/app-packages/command-logger/slider-pkg/package/scripts/params.py
new file mode 100644
index 0000000..3d388ae
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/package/scripts/params.py
@@ -0,0 +1,31 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+from resource_management import *
+
+# server configurations
+config = Script.get_config()
+
+container_id = config['hostLevelParams']['container_id']
+application_id = config['configurations']['global']['application_id']
+app_user = config['configurations']['global']['app_user']
+
+datetime_format = config['configurations']['cl-site']['datetime.format']
+file_location = config['configurations']['cl-site']['logfile.location']

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/package/templates/operations.log.j2
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/package/templates/operations.log.j2 b/app-packages/command-logger/slider-pkg/package/templates/operations.log.j2
new file mode 100644
index 0000000..8d6e879
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/package/templates/operations.log.j2
@@ -0,0 +1,22 @@
+{#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#}
+
+This is a log for all operations on a Yarn container.
+Container Id: {{container_id}}
+Application id: {{application_id}}
+---------------

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/pom.xml b/app-packages/command-logger/slider-pkg/pom.xml
new file mode 100644
index 0000000..72af03e
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/pom.xml
@@ -0,0 +1,107 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+  <parent>
+    <groupId>org.apache.slider</groupId>
+    <artifactId>slider</artifactId>
+    <version>0.31.0-incubating-SNAPSHOT</version>
+    <relativePath>../../../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>command-logger-app-package</artifactId>
+  <packaging>pom</packaging>
+  <name>Slider Command Logger App Package</name>
+  <description>Slider Command Logger App Package</description>
+  <properties>
+    <work.dir>package-tmp</work.dir>
+    <app.package.name>apache-slider-command-logger</app.package.name>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <descriptor>src/assembly/command-logger.xml</descriptor>
+          <appendAssemblyId>false</appendAssemblyId>
+          <finalName>${app.package.name}</finalName>
+        </configuration>
+        <executions>
+          <execution>
+            <id>build-app-package</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>${maven-dependency-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>copy-dependencies</id>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeArtifactIds>command-logger</includeArtifactIds>
+              <includeTypes>tar</includeTypes>
+              <outputDirectory>${project.build.directory}/${work.dir}</outputDirectory>
+            </configuration>
+          </execution>
+          <execution>
+            <id>copy</id>
+            <phase>test</phase>
+            <goals>
+              <goal>copy</goal>
+            </goals>
+            <configuration>
+              <artifactItems>
+                <artifactItem>
+                  <groupId>org.apache.slider</groupId>
+                  <artifactId>command-logger</artifactId>
+                  <type>tar</type>
+                  <overWrite>false</overWrite>
+                  <outputDirectory>${project.build.directory}/${work.dir}</outputDirectory>
+                  <destFileName>command-logger.tar</destFileName>
+                </artifactItem>
+              </artifactItems>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>command-logger</artifactId>
+      <version>${project.version}</version>
+      <type>tar</type>
+    </dependency>
+  </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/resources.json
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/resources.json b/app-packages/command-logger/slider-pkg/resources.json
new file mode 100644
index 0000000..8345661
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/resources.json
@@ -0,0 +1,15 @@
+{
+    "schema": "http://example.org/specification/v2.0.0",
+    "metadata": {
+    },
+    "global": {
+    },
+    "components": {
+        "COMMAND_LOGGER": {
+            "yarn.role.priority": "1",
+            "yarn.component.instances": "1"
+        },
+        "slider-appmaster": {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/command-logger/slider-pkg/src/assembly/command-logger.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/src/assembly/command-logger.xml b/app-packages/command-logger/slider-pkg/src/assembly/command-logger.xml
new file mode 100644
index 0000000..cad468a
--- /dev/null
+++ b/app-packages/command-logger/slider-pkg/src/assembly/command-logger.xml
@@ -0,0 +1,78 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~       http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+
+<assembly
+  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>command_logger_app_pkg</id>
+  <formats>
+    <format>zip</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <files>
+    <file>
+      <source>appConfig.json</source>
+      <outputDirectory>/</outputDirectory>
+      <filtered>true</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+    <file>
+      <source>resources.json</source>
+      <outputDirectory>/</outputDirectory>
+      <filtered>true</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+    <file>
+      <source>metainfo.xml</source>
+      <outputDirectory>/</outputDirectory>
+      <filtered>true</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+  </files>
+
+  <fileSets>
+    <fileSet>
+      <directory>${project.build.directory}/${work.dir}</directory>
+      <outputDirectory>package/files</outputDirectory>
+      <includes>
+        <include>command-logger.tar</include>
+      </includes>
+      <fileMode>0755</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
+
+    <fileSet>
+      <directory>${project.basedir}</directory>
+      <outputDirectory>/</outputDirectory>
+      <excludes>
+        <exclude>pom.xml</exclude>
+        <exclude>src/**</exclude>
+        <exclude>target/**</exclude>
+        <exclude>appConfig.json</exclude>
+        <exclude>resources.json</exclude>
+        <exclude>metainfo.xml</exclude>
+      </excludes>
+      <fileMode>0755</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
+
+  </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/app-packages/hbase/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/appConfig.json b/app-packages/hbase/appConfig.json
index 6225129..9a43863 100644
--- a/app-packages/hbase/appConfig.json
+++ b/app-packages/hbase/appConfig.json
@@ -7,11 +7,11 @@
     "application.def": "/slider/${app.package.name}.zip",
     "config_types": "core-site,hdfs-site,hbase-site",
     "java_home": "/usr/jdk64/jdk1.7.0_45",
-    "package_list": "files/hbase-${hbase.version}-hadoop2-bin.tar.gz",
+    "package_list": "files/hbase-${hbase.version}-bin.tar.gz",
     "site.global.app_user": "yarn",
     "site.global.app_log_dir": "app/log",
     "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
-    "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/hbase-${hbase.version}-hadoop2",
+    "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/hbase-${hbase.version}",
     "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
     "site.global.hbase_master_heapsize": "1024m",
     "site.global.hbase_regionserver_heapsize": "1024m",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e879109a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e517063..e1f772b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,6 +34,8 @@
   </organization>
 
   <modules>
+    <module>app-packages/command-logger/application-pkg</module>
+    <module>app-packages/command-logger/slider-pkg</module>
     <module>slider-core</module>
     <module>slider-agent</module>
     <module>app-packages/accumulo</module>


[18/26] git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop

Posted by st...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: e4723f5e538d15c39320c3651d6bf4c400c59400
Parents: 613dbec be94df3
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 21:44:44 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 21:44:44 2014 +0100

----------------------------------------------------------------------
 app-packages/command-logger/README.txt          |  22 ++++
 .../command-logger/application-pkg/README.txt   |  24 ++++
 .../command-logger/application-pkg/pom.xml      |  83 +++++++++++++
 .../src/command_logger/README.txt               |  18 +++
 .../src/command_logger/operations.log           |  24 ++++
 .../src/packages/tarball/all.xml                |  35 ++++++
 .../command-logger/slider-pkg/appConfig.json    |  27 ++++
 .../slider-pkg/configuration/cl-site.xml        |  34 ++++++
 .../command-logger/slider-pkg/metainfo.xml      |  52 ++++++++
 .../slider-pkg/package/scripts/cl.py            |  89 ++++++++++++++
 .../slider-pkg/package/scripts/params.py        |  31 +++++
 .../package/templates/operations.log.j2         |  22 ++++
 app-packages/command-logger/slider-pkg/pom.xml  | 122 +++++++++++++++++++
 .../command-logger/slider-pkg/resources.json    |  15 +++
 .../slider-pkg/src/assembly/command-logger.xml  |  78 ++++++++++++
 app-packages/hbase/README.txt                   |   2 +-
 app-packages/hbase/appConfig.json               |   4 +-
 pom.xml                                         |   2 +
 .../test_command_log/appConfig.json             |   6 +-
 .../test_command_log/appConfig_fast_no_reg.json |   6 +-
 .../test_command_log/appConfig_no_hb.json       |   6 +-
 .../test_command_log/configuration/cl-site.xml  |  34 ------
 .../app_packages/test_command_log/metainfo.xml  |  52 --------
 .../package/files/command_log_10.tar            | Bin 2560 -> 0 bytes
 .../test_command_log/package/scripts/cl.py      |  89 --------------
 .../test_command_log/package/scripts/params.py  |  31 -----
 .../package/templates/operations.log.j2         |  22 ----
 slider-funtest/pom.xml                          |  52 +++++++-
 .../lifecycle/AgentCommandTestBase.groovy       |  21 +---
 29 files changed, 747 insertions(+), 256 deletions(-)
----------------------------------------------------------------------



[02/26] git commit: Merge branch 'develop' into feature/SLIDER-162_test_runner_to_bootstrap_clean_HDFS_cluster_for_test_runs

Posted by st...@apache.org.
Merge branch 'develop' into feature/SLIDER-162_test_runner_to_bootstrap_clean_HDFS_cluster_for_test_runs


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 1bdf134af5dd9fdb8186c848df8806324296f9cc
Parents: e3c5d9d 0ad4fb4
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jun 30 16:18:14 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jun 30 16:18:14 2014 +0100

----------------------------------------------------------------------
 LICENSE                                         | 423 +++++++++++++++++++
 LICENSE.txt                                     | 423 -------------------
 NOTICE                                          |   5 +
 NOTICE.txt                                      |   5 -
 app-packages/accumulo/metainfo.xml              | 238 +++++------
 app-packages/hbase/metainfo.xml                 | 168 ++++----
 app-packages/storm-v0_91/metainfo.xml           | 229 +++++-----
 slider-assembly/src/main/scripts/slider.py      | 107 +++--
 .../providers/agent/AgentClientProvider.java    |  29 +-
 .../slider/providers/agent/AgentKeys.java       |   1 -
 .../providers/agent/AgentProviderService.java   |  53 +--
 .../agent/application/metadata/Application.java | 121 ++++++
 .../agent/application/metadata/Metainfo.java    |  17 +-
 .../application/metadata/MetainfoParser.java    |  10 +-
 .../agent/application/metadata/Service.java     | 121 ------
 .../appmaster/state/ContainerPriority.java      |   8 +-
 .../appmaster/state/OutstandingRequest.java     |   7 +-
 .../server/appmaster/state/RoleHistory.java     |   8 +
 .../app_packages/test_command_log/metainfo.xml  |  62 ++-
 .../agent/AgentMiniClusterTestBase.groovy       |  41 +-
 .../slider/providers/agent/AgentTestBase.groovy |  44 +-
 .../agent/TestAgentAMManagementWS.groovy        |   3 +-
 .../slider/providers/agent/TestAgentEcho.groovy |   6 +-
 .../providers/agent/TestBuildBasicAgent.groovy  |  78 ++--
 .../curator/TestRegistryRestResources.groovy    |   3 +-
 .../model/appstate/TestMockRMOperations.groovy  |   2 +-
 .../publisher/TestPublisherRestResources.groovy |   3 +-
 .../slider/common/tools/TestSliderUtils.java    |  23 +-
 .../agent/TestAgentClientProvider.java          |  40 +-
 .../agent/TestAgentProviderService.java         |  61 ++-
 .../metadata/MetainfoParserTest.java            |  15 +-
 .../org/apache/slider/tools/TestUtility.java    |  71 ++++
 slider-core/src/test/python/appdef_1.zip        | Bin 972 -> 0 bytes
 slider-core/src/test/python/metainfo.xml        |  68 ++-
 .../org/apache/slider/common/tools/test.zip     | Bin 1273 -> 0 bytes
 .../slider/common/tools/test/metainfo.txt       |  16 +
 .../slider/common/tools/test/metainfo.xml       |  95 +++++
 .../slider/common/tools/test/someOtherFile.txt  |  16 +
 .../slider/common/tools/test/someOtherFile.xml  |  16 +
 .../agent/application/metadata/metainfo.xml     | 136 +++---
 .../slider/providers/hbase/HBaseRoles.java      |   3 +-
 41 files changed, 1543 insertions(+), 1232 deletions(-)
----------------------------------------------------------------------



[10/26] git commit: SLIDER-171: connectivity tests to verify state of funtest client config

Posted by st...@apache.org.
SLIDER-171: connectivity tests to verify state of funtest client config


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: fb73dbca80cd87c2880d7e46325a67c507f58ee0
Parents: a6a4680
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 20:08:54 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 20:08:54 2014 +0100

----------------------------------------------------------------------
 .../slider/client/SliderYarnClientImpl.java     |  3 +-
 .../funtest/framework/CommandTestBase.groovy    |  4 +-
 .../basic/TestClusterConnectivity.groovy        | 96 ++++++++++++++++++++
 .../clusters/c6401/slider/slider-client.xml     | 14 ++-
 .../clusters/morzine/slider/slider-client.xml   | 29 ++++--
 5 files changed, 132 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fb73dbca/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 0c83e0c..e7b492b 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
@@ -19,6 +19,7 @@
 package org.apache.slider.client;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
@@ -74,7 +75,7 @@ public class SliderYarnClientImpl extends YarnClientImpl {
     List<ApplicationReport> allApps = getApplications(types);
     List<ApplicationReport> results = new ArrayList<>();
     for (ApplicationReport report : allApps) {
-      if (user == null || user.equals(report.getUser())) {
+      if (StringUtils.isEmpty(user) || user.equals(report.getUser())) {
         results.add(report);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fb73dbca/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 61ae804..08d352a 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -103,15 +103,13 @@ abstract class CommandTestBase extends SliderTestUtils {
       log.debug("Security enabled")
       SliderUtils.forceLogin()
     } else {
-      log.info "Security off, making cluster dirs broadly accessible"
+      log.info "Security is off"
     }
     SliderShell.confDir = SLIDER_CONF_DIRECTORY
     SliderShell.script = SLIDER_SCRIPT
     log.info("Test using ${HadoopFS.getDefaultUri(SLIDER_CONFIG)} " +
              "and YARN RM @ ${SLIDER_CONFIG.get(YarnConfiguration.RM_ADDRESS)}")
 
-    // now patch the settings with the path of the conf direcotry
-
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fb73dbca/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
new file mode 100644
index 0000000..cf9cc9d
--- /dev/null
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.funtest.basic
+
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.slider.client.SliderYarnClientImpl
+import org.apache.slider.common.SliderXmlConfKeys
+import org.apache.slider.core.zk.ZookeeperUtils
+import org.apache.slider.funtest.framework.CommandTestBase
+import org.junit.Test
+
+@Slf4j
+class TestClusterConnectivity extends CommandTestBase {
+
+  @Test
+  public void testFileSystemUp() throws Throwable {
+
+    def fs = clusterFS
+    def status = fs.listStatus(new Path("/"))
+    status.each {it -> log.info(it.path.toString())}
+    
+  }
+
+  @Test
+  public void testZKBinding() throws Throwable {
+    def quorum = SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.REGISTRY_ZK_QUORUM)
+    assert quorum
+    def tuples = ZookeeperUtils.splitToHostsAndPortsStrictly(quorum);
+    tuples.each {it -> telnet(it.getHostText(), it.getPort())}
+    
+  }
+
+  @Test
+  public void testRMTelnet() throws Throwable {
+    def rmAddr = SLIDER_CONFIG.getSocketAddr(YarnConfiguration.RM_ADDRESS, "", 0)
+    telnet(rmAddr.hostName, rmAddr.port)
+  }
+  
+  @Test
+  public void testRMBinding() throws Throwable {
+    testRMTelnet()
+    SliderYarnClientImpl yarnClient = new SliderYarnClientImpl()
+    try {
+      SLIDER_CONFIG.setInt("ipc.client.connect.retry.interval",100)
+      SLIDER_CONFIG.setInt(
+          YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,5000)
+      SLIDER_CONFIG.setInt(
+          YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,50)
+      
+      yarnClient.init(SLIDER_CONFIG)
+      yarnClient.start();
+      def instances = yarnClient.listInstances("")
+      instances.each {it -> log.info("Instance $it.applicationId")}
+    } finally {
+      yarnClient.stop()
+    }
+  }
+  
+  def telnet(String host, int port) {
+    assert host != ""
+    assert port != 0
+    try {
+      def socket = new Socket(host, port);
+    } catch (IOException e) {
+      throw NetUtils.wrapException(host, port, "localhost", 0, e)
+    }
+/*
+
+    socket.withStreams { input, output ->
+      output << "echo testing ...\n"
+      def buffer = input.newReader().readLine()
+      println "response = ${buffer}"
+*/
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fb73dbca/src/test/clusters/c6401/slider/slider-client.xml
----------------------------------------------------------------------
diff --git a/src/test/clusters/c6401/slider/slider-client.xml b/src/test/clusters/c6401/slider/slider-client.xml
index fe2caad..526881b 100644
--- a/src/test/clusters/c6401/slider/slider-client.xml
+++ b/src/test/clusters/c6401/slider/slider-client.xml
@@ -21,25 +21,31 @@
   They are not passed to the AM
 -->
 <configuration>
+  
+  <property>
+    <name>hostname</name>
+    <value>c6401</value>
+  </property>
+  
   <property>
     <name>slider.client.resource.origin</name>
-    <value>configs/c6401/slider</value>
+    <value>configs/${hostname}/slider</value>
     <description>This is just for diagnostics</description>
   </property>
 
   <property>
     <name>yarn.resourcemanager.address</name>
-    <value>c6401:8050</value>
+    <value>${hostname}:8050</value>
   </property>
   
   <property>
     <name>fs.defaultFS</name>
-    <value>hdfs://c6401.ambari.apache.org:8020</value>
+    <value>hdfs://${hostname}.ambari.apache.org:8020</value>
   </property>
 
   <property>
     <name>slider.zookeeper.quorum</name>
-    <value>c6401:2181</value>
+    <value>${hostname}:2181</value>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fb73dbca/src/test/clusters/morzine/slider/slider-client.xml
----------------------------------------------------------------------
diff --git a/src/test/clusters/morzine/slider/slider-client.xml b/src/test/clusters/morzine/slider/slider-client.xml
index d15d443..4d7ab41 100644
--- a/src/test/clusters/morzine/slider/slider-client.xml
+++ b/src/test/clusters/morzine/slider/slider-client.xml
@@ -17,28 +17,45 @@
    limitations under the License.
 -->
 <!--
-  Morzine is runing Hadoop on Windows
+  Properties set here are picked up in the client.
+  They are not passed to the AM
 -->
+<configuration>
+
+  <property>
+    <name>hostname</name>
+    <value>morzine</value>
+  </property>
+
+  <property>
+    <name>slider.client.resource.origin</name>
+    <value>configs/${hostname}/slider</value>
+    <description>This is just for diagnostics</description>
+  </property>
 
   <property>
     <name>yarn.resourcemanager.address</name>
-    <value>morzine:8050</value>
+    <value>${hostname}:8032</value>
   </property>
-  
+
   <property>
     <name>fs.defaultFS</name>
-    <value>hdfs://morzine:8020</value>
+    <value>hdfs://${hostname}:8020</value>
   </property>
 
   <property>
     <name>slider.zookeeper.quorum</name>
-    <value>morzine:2181</value>
+    <value>${hostname}:2181</value>
   </property>
 
+  <!-- 
+  This is a windows path as picked up from
+   http://morzine:8088/conf
+  -->
   <property>
     <name>yarn.application.classpath</name>
     <value>
-      /etc/hadoop/conf,/usr/lib/hadoop/*,/usr/lib/hadoop/lib/*,/usr/lib/hadoop-hdfs/*,/usr/lib/hadoop-hdfs/lib/*,/usr/lib/hadoop-yarn/*,/usr/lib/hadoop-yarn/lib/*,/usr/lib/hadoop-mapreduce/*,/usr/lib/hadoop-mapreduce/lib/*
+      %HADOOP_CONF_DIR%,%HADOOP_COMMON_HOME%/share/hadoop/common/*,%HADOOP_COMMON_HOME%/share/hadoop/common/lib/*,%HADOOP_HDFS_HOME%/share/hadoop/hdfs/*,%HADOOP_HDFS_HOME%/share/hadoop/hdfs/lib/*,%HADOOP_MAPRED_HOME%/share/hadoop/mapreduce/*,%HADOOP_MAPRED_HOME%/share/hadoop/mapreduce/lib/*,%HADOOP_YARN_HOME%/share/hadoop/yarn/*,%HADOOP_YARN_HOME%/share/hadoop/yarn/lib/*
     </value>
   </property>
 


[08/26] git commit: SLIDER-171 create config for windows VM "morzine"

Posted by st...@apache.org.
SLIDER-171 create config for windows VM "morzine"


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 0060d48bda1f18f181029bf49d93be7e25c69e96
Parents: 9a54a0b
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 18:49:16 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 18:49:16 2014 +0100

----------------------------------------------------------------------
 .../clusters/morzine/slider/log4j.properties    | 83 ++++++++++++++++++++
 .../clusters/morzine/slider/slider-client.xml   | 59 ++++++++++++++
 2 files changed, 142 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0060d48b/src/test/clusters/morzine/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/src/test/clusters/morzine/slider/log4j.properties b/src/test/clusters/morzine/slider/log4j.properties
new file mode 100644
index 0000000..6211771
--- /dev/null
+++ b/src/test/clusters/morzine/slider/log4j.properties
@@ -0,0 +1,83 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+#   you may not use this file except in compliance with the License.
+#   You may obtain a copy of the License at
+#   
+#    http://www.apache.org/licenses/LICENSE-2.0
+#   
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License. See accompanying LICENSE file.
+#
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+
+#   Licensed under the Apache License, Version 2.0 (the "License");
+#   you may not use this file except in compliance with the License.
+#   You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=INFO,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+
+# log layout skips stack-trace creation operations by avoiding line numbers and method
+#log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n
+
+# debug edition is much more expensive
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+log4j.appender.subprocess=org.apache.log4j.ConsoleAppender
+log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout
+log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n
+
+log4j.logger.org.apache.slider=DEBUG
+
+# uncomment to debug service lifecycle issues
+#log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG
+#log4j.logger.org.apache.hadoop.yarn.service=DEBUG
+
+# uncomment for YARN operations
+#log4j.logger.org.apache.hadoop.yarn.client=DEBUG
+
+# uncomment this to debug security problems
+#log4j.logger.org.apache.hadoop.security=DEBUG
+
+#crank back on some noise
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.hdfs=WARN
+
+
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl=WARN
+log4j.logger.org.apache.zookeeper=WARN
+
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0060d48b/src/test/clusters/morzine/slider/slider-client.xml
----------------------------------------------------------------------
diff --git a/src/test/clusters/morzine/slider/slider-client.xml b/src/test/clusters/morzine/slider/slider-client.xml
new file mode 100644
index 0000000..d15d443
--- /dev/null
+++ b/src/test/clusters/morzine/slider/slider-client.xml
@@ -0,0 +1,59 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<!--
+  Morzine is runing Hadoop on Windows
+-->
+
+  <property>
+    <name>yarn.resourcemanager.address</name>
+    <value>morzine:8050</value>
+  </property>
+  
+  <property>
+    <name>fs.defaultFS</name>
+    <value>hdfs://morzine:8020</value>
+  </property>
+
+  <property>
+    <name>slider.zookeeper.quorum</name>
+    <value>morzine:2181</value>
+  </property>
+
+  <property>
+    <name>yarn.application.classpath</name>
+    <value>
+      /etc/hadoop/conf,/usr/lib/hadoop/*,/usr/lib/hadoop/lib/*,/usr/lib/hadoop-hdfs/*,/usr/lib/hadoop-hdfs/lib/*,/usr/lib/hadoop-yarn/*,/usr/lib/hadoop-yarn/lib/*,/usr/lib/hadoop-mapreduce/*,/usr/lib/hadoop-mapreduce/lib/*
+    </value>
+  </property>
+
+  <property>
+    <name>slider.test.agent.enabled</name>
+    <description>Flag to enable/disable Agent tests</description>
+    <value>true</value>
+  </property>
+
+
+  <property>
+    <name>slider.test.am.restart.time</name>
+    <description>Time in millis to await an AM restart</description>
+    <value>60000</value>
+  </property>
+
+
+</configuration>


[11/26] git commit: SLIDER-171 more details on dir owner for diags

Posted by st...@apache.org.
SLIDER-171 more details on dir owner for diags


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: abb3d0190c55a79224e5b868844f72b82d28d71f
Parents: fb73dbc
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 20:15:51 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 20:15:51 2014 +0100

----------------------------------------------------------------------
 .../org/apache/slider/funtest/basic/TestClusterConnectivity.groovy | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/abb3d019/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
index cf9cc9d..dcc7333 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
@@ -36,7 +36,7 @@ class TestClusterConnectivity extends CommandTestBase {
 
     def fs = clusterFS
     def status = fs.listStatus(new Path("/"))
-    status.each {it -> log.info(it.path.toString())}
+    status.each {it -> log.info("${it.path} = ${it}")}
     
   }
 


[12/26] git commit: Fix typo in app-packages/hbase/README.txt

Posted by st...@apache.org.
Fix typo in app-packages/hbase/README.txt


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: f53db87257f8a600594446f1a60a9614ec65369f
Parents: 9a54a0b
Author: tedyu <yu...@gmail.com>
Authored: Tue Jul 1 12:37:35 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jul 1 12:37:35 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/README.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f53db872/app-packages/hbase/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/hbase/README.txt b/app-packages/hbase/README.txt
index 78d033e..a25ff8c 100644
--- a/app-packages/hbase/README.txt
+++ b/app-packages/hbase/README.txt
@@ -30,7 +30,7 @@ Use the following command to install HBase tarball locally:
 After HBase tarball is publised locally in maven repository, you can use the following command:
   mvn clean package -DskipTests -Phbase-app-package
 App package can be found in
-  app-packages/HBase/target/apache-slider-hbase-${hbase.version}-app-package-${slider.version}.zip
+  app-packages/hbase/target/apache-slider-hbase-${hbase.version}-app-package-${slider.version}.zip
 
 Create a zip package at the root of the package (<slider enlistment>/app-packages/hbase/)
   zip -r hbase-v098.zip .


[26/26] git commit: SLIDER-151 adding actions

Posted by st...@apache.org.
SLIDER-151 adding actions


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 621a981cb185f76917b8c612b4062200208caa45
Parents: fe513d2
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jun 20 16:08:08 2014 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 17:00:06 2014 +0100

----------------------------------------------------------------------
 .../web/rest/management/ManagementResource.java | 43 ++++++++++++++++++--
 1 file changed, 40 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/621a981c/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
index 02199cf..8a6b928 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
@@ -29,6 +29,7 @@ import org.slf4j.LoggerFactory;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.GET;
+import javax.ws.rs.*;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
@@ -46,6 +47,8 @@ import java.net.URL;
 public class ManagementResource extends AbstractSliderResource {
   protected static final Logger log =
       LoggerFactory.getLogger(ManagementResource.class);
+  public static final String CONFIG = "config";
+  public static final String APP_UNDER_MANAGEMENT = "/app";
 
   public ManagementResource(WebAppApi slider) {
     super(slider);
@@ -80,9 +83,9 @@ public class ManagementResource extends AbstractSliderResource {
   }
 
   @GET
-  @Path("/app/configurations/{config}")
+  @Path(APP_UNDER_MANAGEMENT+"/configurations/{config}")
   @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-  public ConfTreeResource getConfTreeResource(@PathParam("config") String config,
+  public ConfTreeResource getConfTreeResource(@PathParam(CONFIG) String config,
                                               @Context UriInfo uriInfo,
                                               @Context HttpServletResponse res) {
     init(res);
@@ -90,11 +93,45 @@ public class ManagementResource extends AbstractSliderResource {
         ResourceFactory.createAggregateConfResource(getAggregateConf(),
       uriInfo.getBaseUriBuilder()
       .path(RestPaths.SLIDER_CONTEXT_ROOT)
-      .path(RestPaths.MANAGEMENT + "/app"));
+      .path(RestPaths.MANAGEMENT + APP_UNDER_MANAGEMENT));
     return aggregateConf.getConfTree(config);
   }
 
   protected AggregateConf getAggregateConf() {
     return slider.getAppState().getInstanceDefinitionSnapshot();
   }
+  
+  @POST
+  @Path("actions/flex") 
+    public void actionFlex() { 
+  }
+  
+  @POST
+  @Path("actions/stop") 
+    public void actionStop() { 
+  }
+  
+    
+  @POST
+  @Path("actions/test/suicide") 
+    public void actionSuicide() { 
+  }
+    
+  @POST
+  @Path("actions/test/kill-container") 
+    public void actionKillContainer() { 
+  }
+
+
+  @GET
+  @Path("containers"+"/components" +"/{name}")
+  public void actionListContainers() {
+  }
+  
+  @GET
+  @Path("containers"+"/components" +"/{name}")
+  public void actionListContainersbyComponent() {
+  }
+
+
 }


[06/26] git commit: SLIDER-192: use hadoop.security.authentication type as indicate whether to use security or not.

Posted by st...@apache.org.
SLIDER-192: use hadoop.security.authentication type as indicate whether to use security or not.


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 8fe71f06738321525501b7c13a50046bf85e43fb
Parents: 31d22b8
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 15:24:58 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 15:24:58 2014 +0100

----------------------------------------------------------------------
 .../java/org/apache/slider/client/SliderClient.java  |  4 +++-
 .../org/apache/slider/common/SliderXmlConfKeys.java  |  4 ++--
 .../org/apache/slider/common/tools/SliderUtils.java  | 15 ++++++++-------
 3 files changed, 13 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8fe71f06/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 95c120c..a337533 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
@@ -894,7 +894,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     if (clusterSecure) {
       // if the cluster is secure, make sure that
       // the relevant security settings go over
-      addConfOptionToCLI(commandLine, config, KEY_SECURITY_ENABLED);
+/*
+      addConfOptionToCLI(commandLine, config, KEY_SECURITY);
+*/
       addConfOptionToCLI(commandLine,
           config,
           DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8fe71f06/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
index 272ae6a..c7b8ea5 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
@@ -93,8 +93,8 @@ public interface SliderXmlConfKeys {
    * Flag which is set to indicate that security should be enabled
    * when talking to this cluster.
    */
-  String KEY_SECURITY_ENABLED =
-      CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
+  String KEY_SECURITY =
+      CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 
   /**
    * queue name

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8fe71f06/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 b97ff63..5479b54 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
@@ -904,7 +904,8 @@ public final class SliderUtils {
    * @return true if the slider client/service should be in secure mode
    */
   public static boolean isHadoopClusterSecure(Configuration conf) {
-    return conf.getBoolean(SliderXmlConfKeys.KEY_SECURITY_ENABLED, false);
+    return SecurityUtil.getAuthenticationMethod(conf) !=
+           UserGroupInformation.AuthenticationMethod.SIMPLE;
   }
 
   /**
@@ -953,22 +954,22 @@ public final class SliderUtils {
         conf.get(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION));
     log.debug("hadoop.security.authorization={}",
         conf.get(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION));
-    SecurityUtil.setAuthenticationMethod(
-        UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
+/*    SecurityUtil.setAuthenticationMethod(
+        UserGroupInformation.AuthenticationMethod.KERBEROS, conf);*/
     UserGroupInformation.setConfiguration(conf);
     UserGroupInformation authUser = UserGroupInformation.getCurrentUser();
     log.debug("Authenticating as " + authUser.toString());
     log.debug("Login user is {}", UserGroupInformation.getLoginUser());
     if (!UserGroupInformation.isSecurityEnabled()) {
       throw new BadConfigException("Although secure mode is enabled," +
-                                   "the application has already set up its user as an insecure entity %s",
-                                   authUser);
+               "the application has already set up its user as an insecure entity %s",
+               authUser);
     }
     if (authUser.getAuthenticationMethod() ==
         UserGroupInformation.AuthenticationMethod.SIMPLE) {
       throw new BadConfigException("Auth User is not Kerberized %s" +
-                   " -security has already been set up with the wrong authentication method",
-                                   authUser);
+       " -security has already been set up with the wrong authentication method",
+                       authUser);
 
     }
 


[15/26] git commit: SLIDER-171 changes to setting up java command and classpath needed to work with cross-platform client

Posted by st...@apache.org.
SLIDER-171 changes to setting up java command and classpath needed to work with cross-platform client


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 853a979006d5a08734a63ff80ce9c15ab577d61e
Parents: abb3d01
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 21:09:51 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 21:09:51 2014 +0100

----------------------------------------------------------------------
 .../apache/slider/common/tools/SliderUtils.java |  2 +-
 .../core/launch/ClasspathConstructor.java       |  4 +--
 .../core/launch/JavaCommandLineBuilder.java     |  2 +-
 .../funtest/framework/FileUploader.groovy       | 30 ++++++++++++++------
 .../lifecycle/TestAgentClusterLifecycle.groovy  |  3 +-
 5 files changed, 28 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/853a9790/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 5479b54..62e61c9 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
@@ -1146,7 +1146,7 @@ public final class SliderUtils {
       // and nothing else
       classpath.appendAll(classpath.javaVMClasspath());
     } else {
-      classpath.addLibDir("./" + libdir);
+      classpath.addLibDir(libdir);
       if (sliderConfDir != null) {
         classpath.addClassDirectory(sliderConfDir);
       }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/853a9790/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
index ee4c4d1..ca325d1 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
@@ -38,8 +38,8 @@ import java.util.List;
  */
 public class ClasspathConstructor {
 
-  //  public static final String CLASS_PATH_SEPARATOR = ApplicationConstants.CLASS_PATH_SEPARATOR;
-  public static final String CLASS_PATH_SEPARATOR = File.pathSeparator;
+    public static final String CLASS_PATH_SEPARATOR = ApplicationConstants.CLASS_PATH_SEPARATOR;
+//  public static final String CLASS_PATH_SEPARATOR = File.pathSeparator;
   private final List<String> pathElements = new ArrayList<>();
 
   public ClasspathConstructor() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/853a9790/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 e8b6eb1..7b60461 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
@@ -38,7 +38,7 @@ public class JavaCommandLineBuilder extends CommandLineBuilder {
    * @return the path to the Java binary
    */
   protected String getJavaBinary() {
-    return ApplicationConstants.Environment.JAVA_HOME.$() + "/bin/java";
+    return ApplicationConstants.Environment.JAVA_HOME.$$() + "/bin/java";
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/853a9790/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
index 2dc85be..921adbf 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FileUploader.groovy
@@ -18,6 +18,7 @@
 
 package org.apache.slider.funtest.framework
 
+import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileSystem as HadoopFS
@@ -29,6 +30,7 @@ import org.apache.hadoop.security.UserGroupInformation
 
 @SuppressWarnings("GroovyOctalInteger")
 @Slf4j
+@CompileStatic
 class FileUploader {
   final Configuration conf
   final UserGroupInformation user
@@ -69,8 +71,8 @@ class FileUploader {
         return FileUtil.copy(src, fs, destPath, false, conf)
       } catch (AccessControlException ace) {
         log.error("No write access to destination directory $dir" +
-                  "Ensure home directory exists and has correct permissions." +
-                  ace, ace)
+                  "Ensure home directory exists and has correct permissions. $ace",
+                  ace)
         throw ace
       }
     } else {
@@ -103,9 +105,10 @@ class FileUploader {
     }
   }
 
-  public def getFileSystemAsUserHdfs() {
-    def hdfs = UserGroupInformation.createRemoteUser("hdfs")
-    getFileSystem(hdfs, HadoopFS.getDefaultUri(conf))
+  public def getFileSystemAsUserName(String username) {
+
+    def user = UserGroupInformation.createRemoteUser(username)
+    getFileSystem(user, HadoopFS.getDefaultUri(conf))
   }
 
   /**
@@ -127,11 +130,22 @@ class FileUploader {
           throw ace;
         }
         //now create as hdfs
-        def FsAsUserHDFS = fileSystemAsUserHdfs
-        FsAsUserHDFS.mkdirs(home, new FsPermission((short) 00755))
-        FsAsUserHDFS.setOwner(home, user.userName, user.primaryGroupName)
+        try {
+          attemptToCreateHomeDir("hdfs", home)
+        } catch (AccessControlException ace2) {
+
+          log.info("Failed to mkdir $home as $user -impersonating 'hadoop'")
+          attemptToCreateHomeDir("hadoop", home)
+
+        }
       }
     }
     return home
   }
+
+  public void attemptToCreateHomeDir(String username, Path home) {
+    def privilegedFS = getFileSystemAsUserName(username)
+    privilegedFS.mkdirs(home, new FsPermission((short) 00755))
+    privilegedFS.setOwner(home, user.userName, user.primaryGroupName)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/853a9790/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
index 06809cc..25a4134 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
@@ -36,7 +36,7 @@ import org.junit.Test
 @CompileStatic
 @Slf4j
 public class TestAgentClusterLifecycle extends AgentCommandTestBase
-implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
+  implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
 
   static String CLUSTER = "test_agent_cluster_lifecycle"
@@ -47,6 +47,7 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
   @Before
   public void prepareCluster() {
     setupCluster(CLUSTER)
+    describe("Create a 0-role cluster, so testing AM start/stop")
   }
 
   @After


[21/26] git commit: SLIDER-200 double "mvn install" failing as rat rejects hbase/target/rat.txt

Posted by st...@apache.org.
SLIDER-200 double "mvn install" failing as rat rejects hbase/target/rat.txt


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 0151b9f53cdc46e39d80709c8090308f53bbfd3b
Parents: e9ddcf0
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 13:30:26 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 13:30:26 2014 +0100

----------------------------------------------------------------------
 pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0151b9f5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 460fb0d..0076561 100644
--- a/pom.xml
+++ b/pom.xml
@@ -300,6 +300,7 @@
             <exclude>**/httpfs-signature.secret</exclude>
             <exclude>**/dfs.exclude</exclude>
             <exclude>**/*.iml</exclude>
+            <exclude>**/rat.txt</exclude>
             <exclude>DISCLAIMER</exclude>
           </excludes>
         </configuration>


[07/26] git commit: SLIDER-159: jenkins testEcho intermittently failing

Posted by st...@apache.org.
SLIDER-159: jenkins testEcho intermittently failing


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 9a54a0b8d3e6d5f5f706407dfe4ba432419b90d2
Parents: 8fe71f0
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 16:19:58 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 16:19:58 2014 +0100

----------------------------------------------------------------------
 .../services/workflow/ForkedProcessService.java       | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9a54a0b8/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
index 7e73005..ee68aa4 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
@@ -281,12 +281,15 @@ public class ForkedProcessService extends AbstractWorkflowExecutorService implem
     }
     long start = System.currentTimeMillis();
     while (System.currentTimeMillis() - start <= duration) {
-      if (finalOutput && process.isFinalOutputProcessed()) {
-        //end of stream, break
-        break;
-      }
-      if (!process.isRecentOutputEmpty()) {
+      boolean finished;
+      if (finalOutput) {
+        // final flag means block until all data is done
+        finished = process.isFinalOutputProcessed();
+      } else {
         // there is some output
+        finished = !process.isRecentOutputEmpty();
+      }
+      if (finished) {
         break;
       }
       try {
@@ -299,5 +302,4 @@ public class ForkedProcessService extends AbstractWorkflowExecutorService implem
     return process.getRecentOutput();
   }
   
-  
 }


[03/26] git commit: Merge branch 'feature/SLIDER-162_test_runner_to_bootstrap_clean_HDFS_cluster_for_test_runs' into develop

Posted by st...@apache.org.
Merge branch 'feature/SLIDER-162_test_runner_to_bootstrap_clean_HDFS_cluster_for_test_runs' into develop


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 3fb598fba45b08b1d1d13124bf3ce5a0b2c77bb5
Parents: 0ad4fb4 1bdf134
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jun 30 16:36:17 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jun 30 16:36:17 2014 +0100

----------------------------------------------------------------------
 .../funtest/framework/AgentUploads.groovy       | 13 ++---
 .../funtest/framework/FileUploader.groovy       | 51 +++++++++++++++++---
 .../lifecycle/AgentCommandTestBase.groovy       | 10 +++-
 3 files changed, 57 insertions(+), 17 deletions(-)
----------------------------------------------------------------------



[14/26] git commit: SLIDER-190. Add projects to create command logger package as well as Slider app package

Posted by st...@apache.org.
SLIDER-190. Add projects to create command logger package as well as Slider app package


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: be94df3b49051dd929dd6dbb2736f9a0daac73b4
Parents: e879109
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jul 1 11:25:30 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 1 12:56:33 2014 -0700

----------------------------------------------------------------------
 .../command-logger/application-pkg/README.txt   |  24 +++++
 .../command-logger/slider-pkg/metainfo.xml      |   2 +-
 app-packages/command-logger/slider-pkg/pom.xml  |  17 +++-
 .../test_command_log/appConfig.json             |   6 +-
 .../test_command_log/appConfig_fast_no_reg.json |   6 +-
 .../test_command_log/appConfig_no_hb.json       |   6 +-
 .../test_command_log/configuration/cl-site.xml  |  34 -------
 .../app_packages/test_command_log/metainfo.xml  |  52 -----------
 .../package/files/command_log_10.tar            | Bin 2560 -> 0 bytes
 .../test_command_log/package/scripts/cl.py      |  89 -------------------
 .../test_command_log/package/scripts/params.py  |  31 -------
 .../package/templates/operations.log.j2         |  22 -----
 slider-funtest/pom.xml                          |  52 ++++++++++-
 .../lifecycle/AgentCommandTestBase.groovy       |  21 ++---
 14 files changed, 107 insertions(+), 255 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/app-packages/command-logger/application-pkg/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/README.txt b/app-packages/command-logger/application-pkg/README.txt
new file mode 100644
index 0000000..5596f30
--- /dev/null
+++ b/app-packages/command-logger/application-pkg/README.txt
@@ -0,0 +1,24 @@
+<!---
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+Command logger is a simple application that does not have any moving parts yet.
+Its sole purpose is to get packaged as a tarball that includes some static files
+suggesting how commands may be logged.
+
+TODO:
+* Add a daemon that gets activated when application is STARTED
+* Provide hooks to validate that daemon is behaving as expected
+* Provide hooks to inject failures

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/app-packages/command-logger/slider-pkg/metainfo.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/metainfo.xml b/app-packages/command-logger/slider-pkg/metainfo.xml
index 845ad93..e17413d 100644
--- a/app-packages/command-logger/slider-pkg/metainfo.xml
+++ b/app-packages/command-logger/slider-pkg/metainfo.xml
@@ -42,7 +42,7 @@
         <packages>
           <package>
             <type>tarball</type>
-            <name>files/command_log.tar.gz</name>
+            <name>files/command_log.tar</name>
           </package>
         </packages>
       </osSpecific>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/app-packages/command-logger/slider-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/pom.xml b/app-packages/command-logger/slider-pkg/pom.xml
index 72af03e..42ec8da 100644
--- a/app-packages/command-logger/slider-pkg/pom.xml
+++ b/app-packages/command-logger/slider-pkg/pom.xml
@@ -24,7 +24,7 @@
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
-  <artifactId>command-logger-app-package</artifactId>
+  <artifactId>apache-slider-command-logger</artifactId>
   <packaging>pom</packaging>
   <name>Slider Command Logger App Package</name>
   <description>Slider Command Logger App Package</description>
@@ -92,6 +92,21 @@
           </execution>
         </executions>
       </plugin>
+
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <version>${apache-rat-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>check-licenses</id>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/appConfig.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/appConfig.json b/slider-core/src/test/app_packages/test_command_log/appConfig.json
index bd76c05..e7f9700 100644
--- a/slider-core/src/test/app_packages/test_command_log/appConfig.json
+++ b/slider-core/src/test/app_packages/test_command_log/appConfig.json
@@ -4,15 +4,15 @@
     },
     "global": {
         "agent.conf": "agent.ini",
-        "application.def": "cmd_log_app_pkg.zip",
+        "application.def": "apache-slider-command-logger.zip",
         "config_types": "cl-site",
         "java_home": "/usr/jdk64/jdk1.7.0_45",
-        "package_list": "files/command_log_10.tar",
+        "package_list": "files/command-logger.tar",
         "site.global.app_user": "yarn",
         "site.global.application_id": "CommandLogger",
         "site.global.app_log_dir": "${AGENT_LOG_ROOT}/app/log",
         "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
-        "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/hbase-0.96.1-hadoop2",
+        "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/command-logger",
         "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
         "site.cl-site.logfile.location": "${AGENT_LOG_ROOT}/app/log/operations.log",
         "site.cl-site.datetime.format": "%A, %d. %B %Y %I:%M%p"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/appConfig_fast_no_reg.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/appConfig_fast_no_reg.json b/slider-core/src/test/app_packages/test_command_log/appConfig_fast_no_reg.json
index 1f495c6..57c935c 100644
--- a/slider-core/src/test/app_packages/test_command_log/appConfig_fast_no_reg.json
+++ b/slider-core/src/test/app_packages/test_command_log/appConfig_fast_no_reg.json
@@ -6,15 +6,15 @@
         "heartbeat.monitor.interval": "20000",
         "agent.instance.debug.data": "ANY:DO_NOT_REGISTER:NONE",
         "agent.conf": "agent.ini",
-        "application.def": "cmd_log_app_pkg.zip",
+        "application.def": "apache-slider-command-logger.zip",
         "config_types": "cl-site",
         "java_home": "/usr/jdk64/jdk1.7.0_45",
-        "package_list": "files/command_log_10.tar",
+        "package_list": "files/command-logger.tar",
         "site.global.app_user": "yarn",
         "site.global.application_id": "CommandLogger",
         "site.global.app_log_dir": "${AGENT_LOG_ROOT}/app/log",
         "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
-        "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/hbase-0.96.1-hadoop2",
+        "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/command-logger",
         "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
         "site.cl-site.logfile.location": "${AGENT_LOG_ROOT}/app/log/operations.log",
         "site.cl-site.datetime.format": "%A, %d. %B %Y %I:%M%p"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/appConfig_no_hb.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/appConfig_no_hb.json b/slider-core/src/test/app_packages/test_command_log/appConfig_no_hb.json
index c824e03..e028140 100644
--- a/slider-core/src/test/app_packages/test_command_log/appConfig_no_hb.json
+++ b/slider-core/src/test/app_packages/test_command_log/appConfig_no_hb.json
@@ -6,15 +6,15 @@
         "heartbeat.monitor.interval": "20000",
         "agent.instance.debug.data": "ANY:DO_NOT_HEARTBEAT:DO_NOT_HEARTBEAT:NONE",
         "agent.conf": "agent.ini",
-        "application.def": "cmd_log_app_pkg.zip",
+        "application.def": "apache-slider-command-logger.zip",
         "config_types": "cl-site",
         "java_home": "/usr/jdk64/jdk1.7.0_45",
-        "package_list": "files/command_log_10.tar",
+        "package_list": "files/command-logger.tar",
         "site.global.app_user": "yarn",
         "site.global.application_id": "CommandLogger",
         "site.global.app_log_dir": "${AGENT_LOG_ROOT}/app/log",
         "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
-        "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/hbase-0.96.1-hadoop2",
+        "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/command-logger",
         "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
         "site.cl-site.logfile.location": "${AGENT_LOG_ROOT}/app/log/operations.log",
         "site.cl-site.datetime.format": "%A, %d. %B %Y %I:%M%p"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/configuration/cl-site.xml
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/configuration/cl-site.xml b/slider-core/src/test/app_packages/test_command_log/configuration/cl-site.xml
deleted file mode 100644
index 977c8bc..0000000
--- a/slider-core/src/test/app_packages/test_command_log/configuration/cl-site.xml
+++ /dev/null
@@ -1,34 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
--->
-<configuration>
-  <property>
-    <name>logfile.location</name>
-    <value>/tmp/logfiles/operations.log</value>
-    <description>The location where the log file is stored.</description>
-  </property>
-  <property>
-    <name>datetime.format</name>
-    <value>%A, %d. %B %Y %I:%M%p</value>
-    <description>The format to use.
-    </description>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/metainfo.xml
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/metainfo.xml b/slider-core/src/test/app_packages/test_command_log/metainfo.xml
deleted file mode 100644
index 845ad93..0000000
--- a/slider-core/src/test/app_packages/test_command_log/metainfo.xml
+++ /dev/null
@@ -1,52 +0,0 @@
-<?xml version="1.0"?>
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-<metainfo>
-  <schemaVersion>2.0</schemaVersion>
-  <application>
-    <name>TEST_COMMAND_LOG</name>
-    <comment>
-      When started it creates a new log file and stores all commands in the
-      log file. When stopped it renames the file.
-    </comment>
-    <version>0.1.0</version>
-    <components>
-      <component>
-        <name>COMMAND_LOGGER</name>
-        <category>MASTER</category>
-        <commandScript>
-          <script>scripts/cl.py</script>
-          <scriptType>PYTHON</scriptType>
-          <timeout>600</timeout>
-        </commandScript>
-      </component>
-    </components>
-
-    <osSpecifics>
-      <osSpecific>
-        <osType>any</osType>
-        <packages>
-          <package>
-            <type>tarball</type>
-            <name>files/command_log.tar.gz</name>
-          </package>
-        </packages>
-      </osSpecific>
-    </osSpecifics>
-
-  </application>
-</metainfo>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/package/files/command_log_10.tar
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/package/files/command_log_10.tar b/slider-core/src/test/app_packages/test_command_log/package/files/command_log_10.tar
deleted file mode 100644
index b8231d1..0000000
Binary files a/slider-core/src/test/app_packages/test_command_log/package/files/command_log_10.tar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/package/scripts/cl.py
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/package/scripts/cl.py b/slider-core/src/test/app_packages/test_command_log/package/scripts/cl.py
deleted file mode 100644
index 6b18faa..0000000
--- a/slider-core/src/test/app_packages/test_command_log/package/scripts/cl.py
+++ /dev/null
@@ -1,89 +0,0 @@
-#!/usr/bin/env python
-"""
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-"""
-
-import sys
-import os
-from datetime import datetime
-from resource_management import *
-from resource_management.core.base import Fail
-
-
-class CommandLogger(Script):
-  def install(self, env):
-    self.install_packages(env)
-
-  def configure(self, env):
-    import params
-
-    env.set_params(params)
-
-  def start(self, env):
-    import params
-
-    env.set_params(params)
-    self.configure(env)
-    self.rename_file(env)
-    self.ensure_file(env)
-    self.check_and_log(env, "Starting instance.")
-
-  def stop(self, env):
-    import params
-
-    env.set_params(params)
-    self.check_and_log(env, "Stopping instance.")
-    self.rename_file(env)
-
-  def status(self, env):
-    Logger.info("Returning status as live.")
-
-  def check_and_log(self, env, message):
-    import params
-
-    file_location = params.file_location
-    datetime_format = params.datetime_format
-    if not os.path.isfile(file_location) or not os.access(file_location,
-                                                          os.W_OK):
-      raise Fail("File does not exist or not writable. %s" % file_location)
-    with open(file_location, "a") as logfile:
-      logfile.write("Time: " + datetime.utcnow().strftime(datetime_format) + "\n")
-      logfile.write("Log: " + message + "\n")
-      logfile.write("---------------\n")
-
-  def rename_file(self, env):
-    import params
-
-    file_location = params.file_location
-    if os.path.isfile(file_location) and \
-      os.access(file_location, os.W_OK):
-      new_file_name = \
-        file_location + "." + datetime.utcnow().strftime("%d_%m_%y_%I_%M_%S")
-      os.rename(file_location, new_file_name)
-
-  def ensure_file(self, env):
-    import params
-
-    file_location = params.file_location
-    TemplateConfig( file_location,
-                    owner = params.app_user,
-                    template_tag = None
-    )
-
-if __name__ == "__main__":
-  CommandLogger().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/package/scripts/params.py b/slider-core/src/test/app_packages/test_command_log/package/scripts/params.py
deleted file mode 100644
index 3d388ae..0000000
--- a/slider-core/src/test/app_packages/test_command_log/package/scripts/params.py
+++ /dev/null
@@ -1,31 +0,0 @@
-#!/usr/bin/env python
-"""
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-"""
-
-from resource_management import *
-
-# server configurations
-config = Script.get_config()
-
-container_id = config['hostLevelParams']['container_id']
-application_id = config['configurations']['global']['application_id']
-app_user = config['configurations']['global']['app_user']
-
-datetime_format = config['configurations']['cl-site']['datetime.format']
-file_location = config['configurations']['cl-site']['logfile.location']

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-core/src/test/app_packages/test_command_log/package/templates/operations.log.j2
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_command_log/package/templates/operations.log.j2 b/slider-core/src/test/app_packages/test_command_log/package/templates/operations.log.j2
deleted file mode 100644
index 8d6e879..0000000
--- a/slider-core/src/test/app_packages/test_command_log/package/templates/operations.log.j2
+++ /dev/null
@@ -1,22 +0,0 @@
-{#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#}
-
-This is a log for all operations on a Yarn container.
-Container Id: {{container_id}}
-Application id: {{application_id}}
----------------

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-funtest/pom.xml
----------------------------------------------------------------------
diff --git a/slider-funtest/pom.xml b/slider-funtest/pom.xml
index 51b5cec..76785a0 100644
--- a/slider-funtest/pom.xml
+++ b/slider-funtest/pom.xml
@@ -27,6 +27,9 @@
     <artifactId>slider</artifactId>
     <version>0.31.0-incubating-SNAPSHOT</version>
   </parent>
+  <properties>
+    <work.dir>package-tmp</work.dir>
+  </properties>
 
   <build>
 
@@ -113,6 +116,7 @@
             <!-- this property must be supplied-->
             <slider.conf.dir>${slider.conf.dir}</slider.conf.dir>
             <slider.bin.dir>../slider-assembly/target/slider-${project.version}-all/slider-${project.version}</slider.bin.dir>
+            <test.app.pkg>target/package-tmp/apache-slider-command-logger.zip</test.app.pkg>
           </systemPropertyVariables>
           <includes>
             <include>**/Test*.java</include>
@@ -143,6 +147,45 @@
           </excludes>
         </configuration>
       </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>${maven-dependency-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>copy-dependencies</id>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeArtifactIds>apache-slider-command-logger</includeArtifactIds>
+              <includeTypes>zip</includeTypes>
+              <outputDirectory>${project.build.directory}/${work.dir}</outputDirectory>
+            </configuration>
+          </execution>
+          <execution>
+            <id>copy</id>
+            <phase>test</phase>
+            <goals>
+              <goal>copy</goal>
+            </goals>
+            <configuration>
+              <artifactItems>
+                <artifactItem>
+                  <groupId>org.apache.slider</groupId>
+                  <artifactId>apache-slider-command-logger</artifactId>
+                  <type>zip</type>
+                  <overWrite>false</overWrite>
+                  <outputDirectory>${project.build.directory}/${work.dir}</outputDirectory>
+                  <destFileName>apache-slider-command-logger.zip</destFileName>
+                </artifactItem>
+              </artifactItems>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
  
     </plugins>
   </build>
@@ -198,7 +241,14 @@
       <artifactId>hadoop-minicluster</artifactId>
       <scope>test</scope>
     </dependency>
-    
+
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>apache-slider-command-logger</artifactId>
+      <version>${project.version}</version>
+      <type>zip</type>
+    </dependency>
+
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/be94df3b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
index 33c0b81..daf102a 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
@@ -39,13 +39,15 @@ class AgentCommandTestBase extends CommandTestBase
 implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
   public static final boolean AGENTTESTS_ENABLED
+  private static String TEST_APP_PKG_PROP = "test.app.pkg"
+
 
   protected static String APP_RESOURCE = "../slider-core/src/test/app_packages/test_command_log/resources.json"
   protected static String APP_TEMPLATE = "../slider-core/src/test/app_packages/test_command_log/appConfig.json"
-  protected static String APP_PKG_DIR = "../slider-core/src/test/app_packages/test_command_log/"
   protected static String AGENT_CONF = "../slider-agent/conf/agent.ini"
-  protected static final File LOCAL_SLIDER_AGENT_TARGZ
   protected static final File LOCAL_AGENT_CONF
+  public static final String TEST_APP_PKG = sysprop(TEST_APP_PKG_PROP)
+
 
   protected static Path agentTarballPath;
   protected static Path appPkgPath;
@@ -83,24 +85,13 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     AgentUploads agentUploads = new AgentUploads(SLIDER_CONFIG)
     agentUploads.uploader.mkHomeDir()
 
-    appPkgPath = new Path(clusterFS.homeDirectory, "cmd_log_app_pkg.zip")
+    appPkgPath = new Path(clusterFS.homeDirectory, "apache-slider-command-logger.zip")
     if (!clusterFS.exists(appPkgPath)) {
       clusterFS.delete(appPkgPath, false)
     }
 
-    def pkgPath = folder.newFolder("testpkg")
-    File zipFileName = new File(pkgPath, "cmd_log_app_pkg.zip").canonicalFile
-
-    def localZipDirectory = new File(APP_PKG_DIR)
-    assume(localZipDirectory.exists(), "App pkg dir not found at $APP_PKG_DIR")
-
-    zipDir(zipFileName.canonicalPath, APP_PKG_DIR)
-
-    // Verify and upload the app pkg
-    assume(zipFileName.exists(), "App pkg not found at $zipFileName")
-    Path localAppPkg = new Path(zipFileName.toURI());
+    File zipFileName = new File(TEST_APP_PKG).canonicalFile
     agentUploads.uploader.copyIfOutOfDate(zipFileName, appPkgPath, false)
-    
   }
 
   public static void logShell(SliderShell shell) {


[17/26] git commit: SLIDER-171: mark up some methods as clearly not for use x-platform, but only for local setup of miniclusters

Posted by st...@apache.org.
SLIDER-171: mark up some methods as clearly not for use x-platform, but only for local setup of miniclusters


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 613dbec829810edff005c9b8aee00eba8f8bc1b8
Parents: 0317c41
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 21:44:35 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 21:44:35 2014 +0100

----------------------------------------------------------------------
 .../org/apache/slider/common/tools/SliderUtils.java |  2 +-
 .../slider/core/launch/ClasspathConstructor.java    | 16 +++++++++++++---
 2 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/613dbec8/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 62e61c9..7c89321 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
@@ -1144,7 +1144,7 @@ public final class SliderUtils {
     if (usingMiniMRCluster) {
       // for mini cluster we pass down the java CP properties
       // and nothing else
-      classpath.appendAll(classpath.javaVMClasspath());
+      classpath.appendAll(classpath.localJVMClasspath());
     } else {
       classpath.addLibDir(libdir);
       if (sliderConfDir != null) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/613dbec8/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
index ca325d1..5b74c03 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
@@ -123,12 +123,12 @@ public class ClasspathConstructor {
 
 
   public void addRemoteClasspathEnvVar() {
-    append(ApplicationConstants.Environment.CLASSPATH.$());
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
   }
 
 
   public void insertRemoteClasspathEnvVar() {
-    append(ApplicationConstants.Environment.CLASSPATH.$());
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
   }
 
 
@@ -152,12 +152,22 @@ public class ClasspathConstructor {
     return dir;
   }
 
+  /**
+   * Split a classpath. This uses the local path separator so MUST NOT
+   * be used to work with remote classpaths
+   * @param localpath local path
+   * @return a splite
+   */
   public Collection<String> splitClasspath(String localpath) {
     String separator = System.getProperty("path.separator");
     return StringUtils.getStringCollection(localpath, separator);
   }
 
-  public Collection<String> javaVMClasspath() {
+  /**
+   * Get the local JVM classpath split up
+   * @return the list of entries on the JVM classpath env var
+   */
+  public Collection<String> localJVMClasspath() {
     return splitClasspath(System.getProperty("java.class.path"));
   }
 


[25/26] git commit: SLIDER-151, drop getNode operation from client and tests

Posted by st...@apache.org.
SLIDER-151, drop getNode operation from client and tests


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: fe513d214736214e0adfd977ff71867f07b2a825
Parents: 5960dd0
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jun 19 14:55:13 2014 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 17:00:06 2014 +0100

----------------------------------------------------------------------
 .../java/org/apache/slider/client/SliderClient.java  | 12 ------------
 .../org/apache/slider/test/SliderTestUtils.groovy    | 15 ---------------
 .../minicluster/HBaseMiniClusterTestBase.groovy      | 15 ---------------
 3 files changed, 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fe513d21/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 a337533..c4bacb7 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
@@ -1820,18 +1820,6 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   /**
-   * Get a node from the AM
-   * @param uuid uuid of node
-   * @return deserialized node
-   * @throws IOException IO problems
-   * @throws NoSuchNodeException if the node isn't found
-   */
-  @VisibleForTesting
-  public ClusterNode getNode(String uuid) throws IOException, YarnException {
-    return createClusterOperations().getNode(uuid);
-  }
-  
-  /**
    * Get the instance definition from the far end
    */
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fe513d21/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
index a250e55..ff1be28 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
@@ -297,21 +297,6 @@ class SliderTestUtils extends Assert {
     return status
   }
 
-  /**
-   * Wait for the hbase master to be live (or past it in the lifecycle)
-   * @param clustername cluster
-   * @param spintime time to wait
-   * @return true if the cluster came out of the sleep time live 
-   * @throws IOException
-   * @throws SliderException
-   */
-  public static boolean spinForClusterStartup(SliderClient client, long spintime,
-      String role)
-      throws WaitTimeoutException, IOException, SliderException {
-    int state = client.waitForRoleInstanceLive(role, spintime);
-    return state == ClusterDescription.STATE_LIVE;
-  }
-
   public static ClusterDescription dumpClusterStatus(SliderClient client, String text) {
     ClusterDescription status = client.clusterDescription;
     dumpClusterDescription(text, status)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fe513d21/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/HBaseMiniClusterTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/HBaseMiniClusterTestBase.groovy b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/HBaseMiniClusterTestBase.groovy
index 3f49771..a6c5e8a 100644
--- a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/HBaseMiniClusterTestBase.groovy
+++ b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/HBaseMiniClusterTestBase.groovy
@@ -108,21 +108,6 @@ public abstract class HBaseMiniClusterTestBase extends YarnZKMiniClusterTestBase
   public void stopAllRegionServers() {
     killJavaProcesses(HREGION, SIGTERM);
   }
-
-
-  public static void assertHBaseMasterNotStopped(SliderClient sliderClient,
-                                          String clustername) {
-    String[] nodes = sliderClient.listNodeUUIDsByRole(ROLE_MASTER);
-    int masterNodeCount = nodes.length;
-    assert masterNodeCount > 0;
-    ClusterNode node = sliderClient.getNode(nodes[0]);
-    if (node.state >= ClusterDescription.STATE_STOPPED) {
-      //stopped, not what is wanted
-      log.error("HBase master has stopped");
-      log.error(node.toString());
-      fail("HBase master has stopped " + node.diagnostics);
-    }
-  }
   
   /**
    * Create an (unshared) HConnection talking to the hbase service that


[19/26] git commit: SLIDER-199. Modify HBase package README to also add a manual way to create .zip packages

Posted by st...@apache.org.
SLIDER-199. Modify HBase package README to also add a manual way to create .zip packages


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 5bb80a42b1f5d6ea0c37d54b9dcb3f2ebf9251e1
Parents: e4723f5
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jul 1 17:42:43 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 1 17:42:48 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/README.txt | 54 +++++++++++++++++++++++++-------------
 pom.xml                       |  1 -
 2 files changed, 36 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5bb80a42/app-packages/hbase/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/hbase/README.txt b/app-packages/hbase/README.txt
index a25ff8c..1d547a2 100644
--- a/app-packages/hbase/README.txt
+++ b/app-packages/hbase/README.txt
@@ -15,40 +15,58 @@
    limitations under the License.
 -->
 
-How to create a Slider package for HBase?
+Create Slider App Package for HBase
+
+While appConfig.json and resources.json are not required for the package they
+work well as the default configuration for Slider apps. So it is advisable that
+when you create an application package for Slider, include sample/default
+resources.json and appConfig.json for a minimal Yarn cluster.
+
+OPTION-I: Use mvn command
+OPTION-II: Manual
+
+****** OPTION - I (use mvn command) **
+You need the HBase version available on local maven repo to create the Slider App Package for HBase.
 
 The version of HBase used for the app package can be adjusted by adding a
 flag such as
   -Dhbase.version=0.98.3
 
-Copy the tarball for HBase:
-  cp ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz package/files/
+Download the tarball for HBase:
+  e.g. path to tarball ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz
 
 Use the following command to install HBase tarball locally:
   mvn install:install-file -Dfile=<path-to-tarball> -DgroupId=org.apache.hbase -DartifactId=hbase -Dversion=0.98.3-hadoop2 -Dclassifier=bin -Dpackaging=tar.gz
 
-After HBase tarball is publised locally in maven repository, you can use the following command:
+After HBase tarball is published locally in maven repository, you can use the following command:
   mvn clean package -DskipTests -Phbase-app-package
+
 App package can be found in
   app-packages/hbase/target/apache-slider-hbase-${hbase.version}-app-package-${slider.version}.zip
 
-Create a zip package at the root of the package (<slider enlistment>/app-packages/hbase/)
-  zip -r hbase-v098.zip .
-
 Verify the content using
   zip -Tv apache-slider-hbase-*.zip
 
-While appConfig.json and resources.json are not required for the package they
-work well as the default configuration for Slider apps. So it is advisable that
-when you create an application package for Slider, include sample/default
-resources.json and appConfig.json for a minimal Yarn cluster.
+If an HBase version older than 0.98.3 is desired, it must be installed in the local maven repo.
 
-If an HBase version older than 0.98.3 is desired, it must be installed in the
-local maven repo.
+A less descriptive file name can be specified with
+  -Dapp.package.name=HBase_98dot3 which would create a file HBase_98dot3.zip.
 
-**Note that the LICENSE.txt and NOTICE.txt that are bundled with the app
-package are designed for HBase 0.98.3 only and may need to be modified to be
-applicable for other versions of the app package.
+****** OPTION - II (manual) **
+The Slider App Package for HBase can also be created manually.
 
-A less descriptive file name can be specified with
--Dapp.package.name=HBase_98dot3 which would create a file HBase_98dot3.zip.
+Download the tarball for HBase:
+  e.g. path to tarball ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz
+
+Copy the hbase tarball to package/files
+  cp ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz package/files
+
+Edit appConfig.json/metainfo.xml
+  Replace 4 occurrences of "${hbase.version}" with the hbase version values such as "0.98.3-hadoop2"
+  Replace 1 occurrence of "${app.package.name}" with the desired app package name, e.g. "hbase-v098"
+
+Create a zip package at the root of the package (<slider enlistment>/app-packages/hbase/)
+  zip -r hbase-v098.zip .
+
+Verify the content using
+  zip -Tv hbase-v098.zip

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5bb80a42/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e1f772b..460fb0d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,7 +39,6 @@
     <module>slider-core</module>
     <module>slider-agent</module>
     <module>app-packages/accumulo</module>
-    <module>app-packages/hbase</module>
     <module>slider-assembly</module>
     <module>slider-funtest</module>
     <module>slider-providers/hbase/slider-hbase-provider</module>


[22/26] git commit: SLIDER-201 jenkins windows support - fail fast on HADOOP-10775

Posted by st...@apache.org.
SLIDER-201 jenkins windows support - fail fast on  HADOOP-10775


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: a6cbab17bf591d7b0a6f908ea291c6b4c549bd6d
Parents: 0151b9f
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 15:37:54 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 15:37:54 2014 +0100

----------------------------------------------------------------------
 .../org/apache/slider/test/YarnMiniClusterTestBase.groovy   | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a6cbab17/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
index e5715b1..a5c8710 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileUtil
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.hdfs.MiniDFSCluster
 import org.apache.hadoop.service.ServiceOperations
+import org.apache.hadoop.util.Shell
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
@@ -48,7 +49,9 @@ import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.main.ServiceLauncherBaseTest
 import org.apache.slider.server.appmaster.SliderAppMaster
 import org.junit.After
+import org.junit.Assert
 import org.junit.Before
+import org.junit.BeforeClass
 import org.junit.Rule
 import org.junit.rules.TestName
 import org.junit.rules.Timeout
@@ -126,6 +129,12 @@ public abstract class YarnMiniClusterTestBase extends ServiceLauncherBaseTest {
           KEY_TEST_TIMEOUT,
           DEFAULT_TEST_TIMEOUT_SECONDS * 1000)
   )
+  @BeforeClass
+  public void checkWindowsSupport() {
+    if (Shell.WINDOWS) {
+      Assert.assertNotNull("winutils.exe not found", Shell.WINUTILS)
+    }
+  } 
 
 
   @Rule


[05/26] git commit: SLIDER-175 Scripts for installing, running, and destroying slider applications

Posted by st...@apache.org.
SLIDER-175 Scripts for installing, running, and destroying slider applications


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 31d22b8884ff83e99050e5e3fea3594ca6c1c862
Parents: 2449f6f
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jun 30 17:57:52 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jun 30 17:57:52 2014 +0100

----------------------------------------------------------------------
 slider-install/README.md                       | 102 ++++++++++++
 slider-install/src/main/bash/slider-client.xml |  83 ++++++++++
 slider-install/src/main/bash/slider_destroy    |  64 ++++++++
 slider-install/src/main/bash/slider_setup      | 173 ++++++++++++++++++++
 slider-install/src/main/bash/slider_setup.conf |  60 +++++++
 5 files changed, 482 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/31d22b88/slider-install/README.md
----------------------------------------------------------------------
diff --git a/slider-install/README.md b/slider-install/README.md
new file mode 100644
index 0000000..a4b7b08
--- /dev/null
+++ b/slider-install/README.md
@@ -0,0 +1,102 @@
+<!---
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+
+slider_setup
+============
+
+Tools for installing, starting, and destroying HBase, Accumulo, and Storm slider apps on YARN.
+
+**WARNING: This is intended for POC/sandbox testing, may not be idempotent so DO NOT use on an existing Production cluster!!!**
+
+Setup
+-----
+1. Clone the repo
+2. Set the necessary cluster variables in `slider_setup.conf`, it shouldn't be necessary to change any other values but thoe ones below
+
+
+    # Zookeeper nodes
+    ZK_QUORUM="zk1:2181,zk2:2181,zk3:2181"
+    
+    # Resource Manager address (yarn.resourcemanager.address)
+    RM_ADDRESS="rm1:8050"
+    
+    # Resource Manager scheduler address (yarn.resourcemanager.scheduler.address)
+    RM_SCHED_ADDRESS="rm1:8030"
+    
+    # Default FS (fs.defaultFS)
+    DEFAULT_FS="hdfs://nn1:8020"
+
+Running
+-------
+* slider_setup is the main script and handles the following
+  1. Pulls down slider and extracts the contents to the SLIDER_INST_DIR
+  2. Modifies slider-client.xml with cluster related info
+  3. Pulls down the slider enabled version of the specified product
+  4. Creates necessary directories and copies required files to HDFS
+  5. For HBase, creates the app dir in HDFS
+  6. Submits the slider base application to the YARN cluster
+
+* The following args are required
+  * -f - The path to the slider_setup.conf that has been modified with cluster info
+  * -p - The product to run (hbase, accumulo, or storm are all that are supported at this time)
+  * -w - The number of "worker" nodes. This has different meaning depending on product.
+    * HBase - number of region servers
+    * Accumulo - number of tablet servers
+    * Storm - number of supervisors
+  * -n - The name of the app, this will be the display name in the resource manager and is used by the teardown process
+
+* HBase Example:
+
+
+    ./slider_setup -f slider_setup.conf -p hbase -w 5 -n hbase-slider
+
+* Accumulo Example:
+
+
+    ./slider_setup -f slider_setup.conf -p accumulo -w 3 -n accumulo-slider
+
+* Storm Example:
+
+
+    ./slider_setup -f slider_setup.conf -p storm -w 3 -n storm-slider
+
+Tear Down
+---------
+
+* slider_destroy will do the following
+  1. Freeze the slider application based on provided name
+  2. Destory the slider application based on provided name
+
+* The following args are required
+  * `-f` - The path to the `slider_setup.conf` that has been modified with cluster info
+  * `-n` - The name of the app, this was provided to the slider_setup tool
+
+* HBase Example:
+
+
+    ./slider_destroy -f slider_setup.conf -n hbase-slider
+
+* Accumulo Example:
+
+
+    ./slider_destroy -f slider_setup.conf -n accumulo-slider
+
+* Storm Example:
+
+
+    ./slider_destroy -f slider_setup.conf -n storm-slider

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/31d22b88/slider-install/src/main/bash/slider-client.xml
----------------------------------------------------------------------
diff --git a/slider-install/src/main/bash/slider-client.xml b/slider-install/src/main/bash/slider-client.xml
new file mode 100644
index 0000000..f7060c1
--- /dev/null
+++ b/slider-install/src/main/bash/slider-client.xml
@@ -0,0 +1,83 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<!--
+  Properties set here are picked up in the client.
+  They are not passed to the AM -though the filesystem
+  binding details (URL And principal) are added to the
+  hbase-site.xml file when a cluster is created.
+-->
+<configuration>
+
+  <property>
+    <name>yarn.log-aggregation-enable</name>
+    <value>true</value>
+  </property>
+  
+
+  <property>
+    <name>slider.yarn.queue</name>
+    <value>default</value>
+    <description>YARN queue for the Application Master</description>
+  </property>
+  
+  <property>
+    <name>yarn.resourcemanager.address</name>
+    <value>@@RM_ADDRESS@@</value>
+  </property>
+
+  <property>
+    <name>yarn.resourcemanager.scheduler.address</name>
+    <value>@@RM_SCHED_ADDRESS@@</value>
+  </property>
+
+  <property>
+    <name>fs.defaultFS</name>
+    <value>@@DEFAULT_FS@@</value>
+  </property>
+
+  <property>
+     <name>yarn.application.classpath</name>
+     <value>@@YARN_CP@@</value>
+  </property>
+
+  <property>
+     <name>slider.zookeeper.quorum</name>
+     <value>@@ZK_QUORUM@@</value>
+  </property>
+
+<!--
+  <property>
+    <name>yarn.resourcemanager.principal</name>
+    <value>yarn/master@MINICLUSTER</value>
+  </property>
+
+  <property>
+    <name>slider.security.enabled</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>dfs.namenode.kerberos.principal</name>
+    <value>hdfs/master@MINICLUSTER</value>
+  </property>
+-->
+
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/31d22b88/slider-install/src/main/bash/slider_destroy
----------------------------------------------------------------------
diff --git a/slider-install/src/main/bash/slider_destroy b/slider-install/src/main/bash/slider_destroy
new file mode 100755
index 0000000..9039751
--- /dev/null
+++ b/slider-install/src/main/bash/slider_destroy
@@ -0,0 +1,64 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#
+# Base Vars
+#
+SCRIPT_NAME=`basename $0`
+SCRIPT_DIR=`cd $(dirname $0) && pwd`
+
+#
+# Functions
+#
+usage() { 
+    echo "Usage: $SCRIPT_NAME -f </path/to/config> -n <app name>"
+    exit 1
+}
+
+#
+# Parse cmd line args
+#
+while getopts "f:n:" opt; do
+    case "$opt" in
+        f) config=$OPTARG;;
+        n) app_name=$OPTARG;;
+        *) usage;;
+    esac
+done
+shift $((OPTIND-1))
+
+if [ -z "$config" ] || [ -z "$app_name" ]; then
+    usage
+fi
+
+#
+# Source the config
+#
+source $config
+
+#
+# Main
+#
+echo -e "\n## Freezing app $app_name"
+sudo -u yarn $SLIDER_INST_DIR/bin/slider freeze $app_name --manager $RM_ADDRESS || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Destroying app $app_name"
+sudo -u yarn $SLIDER_INST_DIR/bin/slider destroy $app_name --manager $RM_ADDRESS || exit 1
+echo "SUCCESS"
+
+exit 0

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/31d22b88/slider-install/src/main/bash/slider_setup
----------------------------------------------------------------------
diff --git a/slider-install/src/main/bash/slider_setup b/slider-install/src/main/bash/slider_setup
new file mode 100755
index 0000000..2c15c95
--- /dev/null
+++ b/slider-install/src/main/bash/slider_setup
@@ -0,0 +1,173 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+#
+# Base Vars
+#
+SCRIPT_NAME=`basename $0`
+SCRIPT_DIR=`cd $(dirname $0) && pwd`
+
+#
+# Functions
+#
+usage() { 
+    echo "Usage: $SCRIPT_NAME -f </path/to/config> -p <product (hbase or storm)> -w <worker count> -n <app name>"
+    exit 1
+}
+
+#
+# Parse cmd line args
+#
+while getopts "f:p:w:n:" opt; do
+    case "$opt" in
+        f) config=$OPTARG;;
+        p) product=$OPTARG;;
+        w) worker_cnt=$OPTARG;;
+        n) app_name=$OPTARG;;
+        *) usage;;
+    esac
+done
+shift $((OPTIND-1))
+
+if [ -z "$config" ] || [ -z "$product" ] || [ -z "$worker_cnt" ] || [ -z "$app_name" ]; then
+    usage
+fi
+
+# Only support know products for now
+if [ $product != "hbase" ] && [ $product != "storm" ] && [ $product != "accumulo"]; then
+   echo "ERROR: Only HBase, Storm, and Accumulo are currently supported"
+   usage
+fi
+
+#
+# Source the config
+#
+source $config
+
+#
+# Product URLs
+#
+if [ $product = "hbase" ]; then
+    app_url=$HBASE_APP_URL
+elif [ $product = "storm" ]; then
+    app_url=$STORM_APP_URL
+elif [ $product = "accumulo" ]; then
+    app_url=$ACCUMULO_APP_URL
+fi
+
+#
+# Main
+#
+echo -e "\n## Creating slider install dir: $SLIDER_INST_DIR"
+mkdir -p $SLIDER_INST_DIR || exit 1
+chown yarn:hadoop $SLIDER_INST_DIR || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Downloading slider from: $SLIDER_URL"
+if [ -f /tmp/${SLIDER_VER}*tar ]; then
+   rm -f /tmp/${SLIDER_VER}*tar
+fi
+cd /tmp && wget $SLIDER_URL || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Extracting slider to $SLIDER_INST_DIR"
+tar -xf /tmp/${SLIDER_VER}*tar --strip-components=1 -C $SLIDER_INST_DIR || exit 1
+chown -R yarn:hadoop $SLIDER_INST_DIR || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Setting conf values"
+(cd $SCRIPT_DIR && cp slider-client.xml $SLIDER_INST_DIR/conf) || exit 1
+sed -i 's|@@RM_ADDRESS@@|'$RM_ADDRESS'|g' $SLIDER_INST_DIR/conf/slider-client.xml || exit 1
+sed -i 's|@@RM_SCHED_ADDRESS@@|'$RM_SCHED_ADDRESS'|g' $SLIDER_INST_DIR/conf/slider-client.xml || exit 1
+sed -i 's|@@DEFAULT_FS@@|'$DEFAULT_FS'|g' $SLIDER_INST_DIR/conf/slider-client.xml || exit 1
+sed -i 's|@@YARN_CP@@|'$YARN_CP'|g' $SLIDER_INST_DIR/conf/slider-client.xml || exit 1
+sed -i 's|@@ZK_QUORUM@@|'$ZK_QUORUM'|g' $SLIDER_INST_DIR/conf/slider-client.xml || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Setting PATH to include the JDK bin: $JDK_BIN"
+export PATH=$PATH:$JDK_BIN
+echo "SUCCESS"
+
+echo -e "\n## Checking version of Hadoop slider was compiled against"
+hadoop_compiled_ver=`$SLIDER_INST_DIR/bin/slider version | grep "Compiled against Hadoop" | awk '{print $NF}'`
+if [ "$hadoop_compiled_ver" != "2.4.0" ]; then
+   echo "ERROR: Compiled against Hadoop version $hadoop_compiled_ver instead of 2.4.0" && exit 1
+else
+   echo "Compiled against Hadoop version: $hadoop_compiled_ver"
+fi
+echo "SUCCESS"
+
+echo -e "\n## Setting up HDFS directories for slider"
+sudo -u hdfs hdfs dfs -mkdir -p /slider || exit 1
+sudo -u hdfs hdfs dfs -chown yarn:hdfs /slider || exit 1
+sudo -u hdfs hdfs dfs -mkdir -p /user/yarn || exit 1
+sudo -u hdfs hdfs dfs -chown yarn:hdfs /user/yarn || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Loading the Slider agent"
+sudo -u yarn hdfs dfs -mkdir -p /slider/agent/conf || exit 1
+sudo -u yarn hdfs dfs -copyFromLocal $SLIDER_INST_DIR/agent/* /slider/agent
+echo "SUCCESS"
+
+echo -e "\n## Downloading $product to $SLIDER_INST_DIR/apps/$product"
+sudo -u yarn mkdir -p $SLIDER_INST_DIR/apps/$product || exit 1
+(cd $SLIDER_INST_DIR/apps/$product && sudo -u yarn wget $app_url) || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Extracting $product in $SLIDER_INST_DIR/apps/$product"
+(cd $SLIDER_INST_DIR/apps/$product && sudo -u yarn unzip -o $product*zip) || exit 1
+echo "SUCCESS"
+
+echo -e "\n## Adding $product to HDFS slider dir"
+sudo -u yarn hdfs dfs -copyFromLocal $SLIDER_INST_DIR/apps/$product/$product*zip /slider
+echo "SUCCESS"
+
+echo -e "\n## Setting number of workers in $SLIDER_INST_DIR/apps/$product/resources.json"
+if [ $product = "hbase" ]; then
+   component="HBASE_REGIONSERVER"
+elif [ $product = "storm" ]; then
+   component="SUPERVISOR"
+elif [ $product = "accumulo" ]; then
+   component="ACCUMULO_TSERVER"
+fi
+# Update the resource.json file with worker_cnt
+python << END
+import json
+with open("$SLIDER_INST_DIR/apps/$product/resources.json", "r+") as f:
+    data = json.load(f)
+    data["components"]["$component"]["yarn.component.instances"] = "$worker_cnt"
+
+with open("$SLIDER_INST_DIR/apps/$product/resources.json", "w+") as f:
+    f.write(json.dumps(data, sort_keys = False, indent = 4))
+END
+echo "SUCCESS"
+
+# Handle HBase HDFS dir needs
+if [ $product = "hbase" ]; then
+    echo -e "\n## Creating hbase HDFS dir /apps/hbase"
+    sudo -u hdfs hdfs dfs -mkdir -p /apps/hbase || exit 1
+    sudo -u hdfs hdfs dfs -chown yarn:hdfs /apps/hbase || exit 1
+    echo "SUCCESS"
+fi
+
+echo -e "\n##Starting app $product with $worker_cnt workers via slider"
+sudo -u yarn $SLIDER_INST_DIR/bin/slider create $app_name \
+    --image $DEFAULT_FS/slider/agent/slider-agent.tar.gz \
+    --template $SLIDER_INST_DIR/apps/$product/appConfig.json \
+    --resources $SLIDER_INST_DIR/apps/$product/resources.json || exit 1
+echo "SUCCESS"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/31d22b88/slider-install/src/main/bash/slider_setup.conf
----------------------------------------------------------------------
diff --git a/slider-install/src/main/bash/slider_setup.conf b/slider-install/src/main/bash/slider_setup.conf
new file mode 100644
index 0000000..fed3f4c
--- /dev/null
+++ b/slider-install/src/main/bash/slider_setup.conf
@@ -0,0 +1,60 @@
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+# Change values below to match the cluster being deployed on
+#
+
+# Zookeeper nodes
+ZK_QUORUM="zk1:2181,zk2:2181,zk3:2181"
+
+# Resource Manager address (yarn.resourcemanager.address)
+RM_ADDRESS="rm1:8050"
+
+# Resource Manager scheduler address (yarn.resourcemanager.scheduler.address)
+RM_SCHED_ADDRESS="rm1:8030"
+
+# Default FS (fs.defaultFS)
+DEFAULT_FS="hdfs://nn1:8020"
+
+
+#
+# Should not be necessary to change anything below except on new version releases
+#
+
+# Slider version
+SLIDER_VER="slider-0.30"
+
+# Slider URL
+SLIDER_URL="http://public-repo-1.hortonworks.com/slider/0.30/slider-0.30-all.tar"
+
+# HBase Slider App URL
+HBASE_APP_URL="http://public-repo-1.hortonworks.com/slider/0.30/apps/hbase_v096.zip"
+
+# Storm Slider App URL
+STORM_APP_URL="http://public-repo-1.hortonworks.com/slider/0.30/apps/storm_v091.zip"
+
+# Accumulo Slider App URL
+ACCUMULO_APP_URL="http://public-repo-1.hortonworks.com/slider/0.30/apps/accumulo_v151.zip"
+
+# Where to install Slider
+SLIDER_INST_DIR="/usr/lib/slider"
+
+# JDK bin to add to PATH
+JDK_BIN="/usr/jdk64/jdk1.7.0_45/bin"
+
+# Yarn container classpath (yarn.application.classpath)
+YARN_CP="/etc/hadoop/conf,/usr/lib/hadoop/*,/usr/lib/hadoop/lib/*,/usr/lib/hadoop-hdfs/*,/usr/lib/hadoop-hdfs/lib/*,/usr/lib/hadoop-yarn/*,/usr/lib/hadoop-yarn/lib/*,/usr/lib/hadoop-mapreduce/*,/usr/lib/hadoop-mapreduce/lib/*"
+


[04/26] git commit: SLIDER-173 Container requests for the same role should utilize different nodes - give higher priority to requests with location specified

Posted by st...@apache.org.
SLIDER-173 Container requests for the same role should utilize different nodes - give higher priority to requests with location specified


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 2449f6f35bee3220d05f9b386487d44439c9a3cc
Parents: 3fb598f
Author: tedyu <yu...@gmail.com>
Authored: Mon Jun 30 09:43:31 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jun 30 09:43:31 2014 -0700

----------------------------------------------------------------------
 .../slider/server/appmaster/state/ContainerPriority.java       | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2449f6f3/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java
index 56a5af2..369a932 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java
@@ -38,11 +38,11 @@ import org.apache.hadoop.yarn.util.Records;
 public final class ContainerPriority {
 
   // bit that represents whether location is specified
-  static final int LOCATION = 1 << 30;
+  static final int NOLOCATION = 1 << 30;
   
   public static int buildPriority(int role,
                                   boolean locationSpecified) {
-    int location = locationSpecified ? LOCATION : 0;
+    int location = locationSpecified ? 0 : NOLOCATION;
     return role | location;
   }
 
@@ -57,7 +57,7 @@ public final class ContainerPriority {
   
   
   public static int extractRole(int priority) {
-    return priority >= LOCATION ? priority^LOCATION : priority;
+    return priority >= NOLOCATION ? priority^NOLOCATION : priority;
   }
 
   /**


[09/26] git commit: SLIDER-192 drop obsolete references to slider.security.enabled

Posted by st...@apache.org.
SLIDER-192 drop obsolete references to slider.security.enabled


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: a6a4680fec58330e5377c05b60061231e0be4037
Parents: 0060d48
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 18:50:01 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 18:50:01 2014 +0100

----------------------------------------------------------------------
 .../clusters/c6401/slider/slider-client.xml     |  5 ---
 .../clusters/offline/slider/slider-client.xml   |  7 +---
 .../clusters/remote/slider/slider-client.xml    |  5 ---
 .../clusters/sandbox/slider/slider-client.xml   |  5 ---
 src/test/clusters/ubuntu-secure/operations.md   | 36 ++++++++++----------
 5 files changed, 19 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a6a4680f/src/test/clusters/c6401/slider/slider-client.xml
----------------------------------------------------------------------
diff --git a/src/test/clusters/c6401/slider/slider-client.xml b/src/test/clusters/c6401/slider/slider-client.xml
index f1aa4fb..fe2caad 100644
--- a/src/test/clusters/c6401/slider/slider-client.xml
+++ b/src/test/clusters/c6401/slider/slider-client.xml
@@ -38,11 +38,6 @@
   </property>
 
   <property>
-    <name>slider.security.enabled</name>
-    <value>false</value>
-  </property>
-
-  <property>
     <name>slider.zookeeper.quorum</name>
     <value>c6401:2181</value>
   </property>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a6a4680f/src/test/clusters/offline/slider/slider-client.xml
----------------------------------------------------------------------
diff --git a/src/test/clusters/offline/slider/slider-client.xml b/src/test/clusters/offline/slider/slider-client.xml
index a49dd29..8385086 100644
--- a/src/test/clusters/offline/slider/slider-client.xml
+++ b/src/test/clusters/offline/slider/slider-client.xml
@@ -41,12 +41,7 @@
     <name>slider.funtest.enabled</name>
     <value>false</value>
   </property>
-  
-  <property>
-    <name>slider.security.enabled</name>
-    <value>false</value>
-  </property>
-  
+
   <property>
     <name>yarn.application.classpath</name>
     <value>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a6a4680f/src/test/clusters/remote/slider/slider-client.xml
----------------------------------------------------------------------
diff --git a/src/test/clusters/remote/slider/slider-client.xml b/src/test/clusters/remote/slider/slider-client.xml
index 5bd2edb..5ed4d10 100644
--- a/src/test/clusters/remote/slider/slider-client.xml
+++ b/src/test/clusters/remote/slider/slider-client.xml
@@ -50,11 +50,6 @@
   </property>
 
   <property>
-    <name>slider.security.enabled</name>
-    <value>false</value>
-  </property>
-
-  <property>
     <name>slider.test.agent.enabled</name>
     <value>true</value>
   </property>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a6a4680f/src/test/clusters/sandbox/slider/slider-client.xml
----------------------------------------------------------------------
diff --git a/src/test/clusters/sandbox/slider/slider-client.xml b/src/test/clusters/sandbox/slider/slider-client.xml
index f15f71d..30937ec 100644
--- a/src/test/clusters/sandbox/slider/slider-client.xml
+++ b/src/test/clusters/sandbox/slider/slider-client.xml
@@ -38,11 +38,6 @@
   </property>
 
   <property>
-    <name>slider.security.enabled</name>
-    <value>false</value>
-  </property>
-
-  <property>
     <name>slider.zookeeper.quorum</name>
     <value>sandbox:2181</value>
   </property>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a6a4680f/src/test/clusters/ubuntu-secure/operations.md
----------------------------------------------------------------------
diff --git a/src/test/clusters/ubuntu-secure/operations.md b/src/test/clusters/ubuntu-secure/operations.md
index cb14dfe..d894038 100644
--- a/src/test/clusters/ubuntu-secure/operations.md
+++ b/src/test/clusters/ubuntu-secure/operations.md
@@ -56,7 +56,7 @@
       --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
          --role workers 4\
           --zkhosts ubuntu --zkport 2121 \
-          -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM \
+          -S java.security.krb5.realm=COTHAM \
           -S java.security.krb5.kdc=ubuntu \
           --image hdfs://ubuntu:9090/hbase.tar \
           --appconf file:////Users/slider/Hadoop/configs/master/hbase \
@@ -70,7 +70,7 @@
     bin/slider create cl1 \
           --provider hbase \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM \
+    -S java.security.krb5.realm=COTHAM \
     -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
             --role worker 1\
@@ -87,7 +87,7 @@
     bin/slider create cl1 \
           --provider hbase \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM \
+    -S java.security.krb5.realm=COTHAM \
     -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM \
@@ -102,14 +102,14 @@
         
     bin/slider status clu1 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM \
+    -S java.security.krb5.realm=COTHAM \
     -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM 
            
     bin/slider list \
     --manager ubuntu:8032 \
-    -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
+    -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
       -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM
                
@@ -122,7 +122,7 @@
           --provider hbase \
     --zkhosts ubuntu --zkport 2121 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
+    -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
     -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
     -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM \
     --image hdfs://ubuntu:9090/hbase.tar \
@@ -138,7 +138,7 @@
       --provider hbase \
       --zkhosts ubuntu  --zkport 2121 \
       --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-      -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
+      -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
       -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
       -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM \
       --image hdfs://ubuntu:9090/hbase.tar \
@@ -151,7 +151,7 @@
           --provider hbase \
          --zkhosts ubuntu  --zkport 2121 \
          --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-         -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
+         -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
          -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
          -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM \
          --image hdfs://ubuntu:9090/hbase.tar \
@@ -166,7 +166,7 @@
       --zkhosts ubuntu \
       --zkport 2121 \
       --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-      -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
+      -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
       -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
       -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM \
       --image hdfs://ubuntu:9090/hbase.tar \
@@ -176,23 +176,23 @@
                
     bin/slider  status cl1 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-     -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
+     -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM 
      
                
-    bin/slider  status cl1 -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu 
+    bin/slider  status cl1 -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu 
     
     
     bin/slider  status cl1 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true \
+    \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM 
      
    bin/slider  status cluster3 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-     -D slider.security.enabled=true \
+     \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM 
      
@@ -200,28 +200,28 @@
                
     bin/slider  thaw cl1 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true \
+    \
      -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM 
                    
     bin/slider  freeze cl1 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true \
+    \
     -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM   
                       
     bin/slider  freeze cluster3 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true \
+    \
     -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM 
     
     bin/slider  destroy cl1 \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true \
+    \
     -S java.security.krb5.realm=COTHAM -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM \
@@ -231,7 +231,7 @@
          
     bin/slider  emergency-force-kill all \
     --manager ubuntu:8032 --filesystem hdfs://ubuntu:9090 \
-    -D slider.security.enabled=true -S java.security.krb5.realm=COTHAM \
+    -S java.security.krb5.realm=COTHAM \
      -S java.security.krb5.kdc=ubuntu \
      -D yarn.resourcemanager.principal=yarn/ubuntu@COTHAM \
      -D dfs.namenode.kerberos.principal=hdfs/ubuntu@COTHAM 


[23/26] git commit: SLIDER-201 jenkins windows support - fail fast wasnt static

Posted by st...@apache.org.
SLIDER-201 jenkins windows support - fail fast wasnt static


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

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 6d96ba95659c2f47771eebbc1c632c89c3ee173f
Parents: a6cbab1
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 16:37:31 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 16:37:31 2014 +0100

----------------------------------------------------------------------
 .../groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6d96ba95/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
index a5c8710..80a83a4 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
@@ -130,9 +130,9 @@ public abstract class YarnMiniClusterTestBase extends ServiceLauncherBaseTest {
           DEFAULT_TEST_TIMEOUT_SECONDS * 1000)
   )
   @BeforeClass
-  public void checkWindowsSupport() {
+  public static void checkWindowsSupport() {
     if (Shell.WINDOWS) {
-      Assert.assertNotNull("winutils.exe not found", Shell.WINUTILS)
+      assertNotNull("winutils.exe not found", Shell.WINUTILS)
     }
   }