You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by jm...@apache.org on 2014/05/12 20:30:31 UTC

svn commit: r1594045 - in /incubator/slider/trunk: app-packages/accumulo-v1_5/ app-packages/hbase-v0_96/ app-packages/storm-v0_91/ slider-core/src/main/java/org/apache/slider/api/ slider-core/src/main/java/org/apache/slider/client/ slider-core/src/main...

Author: jmaron
Date: Mon May 12 18:30:30 2014
New Revision: 1594045

URL: http://svn.apache.org/r1594045
Log:
SLIDER-43 validate resource inputs start with yarn and app conf components do not

Added:
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java
Modified:
    incubator/slider/trunk/app-packages/accumulo-v1_5/resources.json
    incubator/slider/trunk/app-packages/hbase-v0_96/resources.json
    incubator/slider/trunk/app-packages/storm-v0_91/resources.json
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
    incubator/slider/trunk/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json
    incubator/slider/trunk/slider-core/src/test/app_packages/test_command_log/resources.json
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeLoadExamples.groovy
    incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json
    incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/providers/agent/tests/good/resources.json
    incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/resources/org/apache/slider/providers/accumulo/instance/resources.json
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/resources/org/apache/slider/providers/hbase/instance/resources.json
    incubator/slider/trunk/src/site/markdown/getting_started.md
    incubator/slider/trunk/src/site/markdown/slider_specs/creating_app_definitions.md
    incubator/slider/trunk/src/site/markdown/slider_specs/resource_specification.md
    incubator/slider/trunk/src/site/markdown/specification/cli-actions.md

Modified: incubator/slider/trunk/app-packages/accumulo-v1_5/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/app-packages/accumulo-v1_5/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/app-packages/accumulo-v1_5/resources.json (original)
+++ incubator/slider/trunk/app-packages/accumulo-v1_5/resources.json Mon May 12 18:30:30 2014
@@ -6,26 +6,26 @@
   },
   "components": {
     "ACCUMULO_MASTER": {
-      "role.priority": "1",
-      "component.instances": "1"
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1"
     },
     "slider-appmaster": {
     },
     "ACCUMULO_TSERVER": {
-      "role.priority": "2",
-      "component.instances": "1"
+      "yarn.role.priority": "2",
+      "yarn.component.instances": "1"
     },
     "ACCUMULO_MONITOR": {
-      "role.priority": "3",
-      "component.instances": "1"
+      "yarn.role.priority": "3",
+      "yarn.component.instances": "1"
     },
     "ACCUMULO_GC": {
-      "role.priority": "4",
-      "component.instances": "1"
+      "yarn.role.priority": "4",
+      "yarn.component.instances": "1"
     },
     "ACCUMULO_TRACER": {
-      "role.priority": "5",
-      "component.instances": "1"
+      "yarn.role.priority": "5",
+      "yarn.component.instances": "1"
     }
   }
 }

Modified: incubator/slider/trunk/app-packages/hbase-v0_96/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/app-packages/hbase-v0_96/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/app-packages/hbase-v0_96/resources.json (original)
+++ incubator/slider/trunk/app-packages/hbase-v0_96/resources.json Mon May 12 18:30:30 2014
@@ -6,14 +6,14 @@
   },
   "components": {
     "HBASE_MASTER": {
-      "role.priority": "1",
-      "component.instances": "1"
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1"
     },
     "slider-appmaster": {
     },
     "HBASE_REGIONSERVER": {
-      "role.priority": "2",
-      "component.instances": "1"
+      "yarn.role.priority": "2",
+      "yarn.omponent.instances": "1"
     }
   }
 }
\ No newline at end of file

Modified: incubator/slider/trunk/app-packages/storm-v0_91/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/app-packages/storm-v0_91/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/app-packages/storm-v0_91/resources.json (original)
+++ incubator/slider/trunk/app-packages/storm-v0_91/resources.json Mon May 12 18:30:30 2014
@@ -8,24 +8,24 @@
     "slider-appmaster": {
     },
     "NIMBUS": {
-      "role.priority": "1",
-      "component.instances": "1"
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1"
     },
     "STORM_REST_API": {
-      "role.priority": "2",
-      "component.instances": "1"
+      "yarn.role.priority": "2",
+      "yarn.component.instances": "1"
     },
     "STORM_UI_SERVER": {
-      "role.priority": "3",
-      "component.instances": "1"
+      "yarn.role.priority": "3",
+      "yarn.component.instances": "1"
     },
     "DRPC_SERVER": {
-      "role.priority": "4",
-      "component.instances": "1"
+      "yarn.role.priority": "4",
+      "yarn.component.instances": "1"
     },
     "SUPERVISOR": {
-      "role.priority": "5",
-      "component.instances": "1"
+      "yarn.role.priority": "5",
+      "yarn.component.instances": "1"
     }
   }
 }
\ No newline at end of file

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java Mon May 12 18:30:30 2014
@@ -28,7 +28,7 @@ public interface ResourceKeys {
    * #of instances of a component
    *
   */
-  String COMPONENT_INSTANCES = "component.instances";
+  String COMPONENT_INSTANCES = "yarn.component.instances";
 
   /**
    *  Amount of memory to ask YARN for in MB.
@@ -63,10 +63,10 @@ public interface ResourceKeys {
    * 2. this must be >= 1
    * 3. this must not match any other role priority in the cluster.
    */
-  String COMPONENT_PRIORITY = "role.priority";
+  String COMPONENT_PRIORITY = "yarn.role.priority";
   
   /**
    * placement policy
    */
-  String COMPONENT_PLACEMENT_POLICY = "component.placement.policy";
+  String COMPONENT_PLACEMENT_POLICY = "yarn.component.placement.policy";
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java Mon May 12 18:30:30 2014
@@ -67,6 +67,8 @@ import org.apache.slider.core.conf.Aggre
 import org.apache.slider.core.conf.ConfTree;
 import org.apache.slider.core.conf.ConfTreeOperations;
 import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.conf.ResourcesInputPropertiesValidator;
+import org.apache.slider.core.conf.TemplateInputPropertiesValidator;
 import org.apache.slider.core.exceptions.BadClusterStateException;
 import org.apache.slider.core.exceptions.BadCommandArgumentsException;
 import org.apache.slider.core.exceptions.BadConfigException;
@@ -396,7 +398,8 @@ public class SliderClient extends Abstra
     //load in any specified on the command line
     if (buildInfo.resources != null) {
       try {
-        resources.mergeFile(buildInfo.resources);
+        resources.mergeFile(buildInfo.resources,
+                            new ResourcesInputPropertiesValidator());
 
       } catch (IOException e) {
         throw new BadConfigException(e,
@@ -408,7 +411,8 @@ public class SliderClient extends Abstra
     }
     if (buildInfo.template != null) {
       try {
-        appConf.mergeFile(buildInfo.template);
+        appConf.mergeFile(buildInfo.template,
+                          new TemplateInputPropertiesValidator());
       } catch (IOException e) {
         throw new BadConfigException(e,
                                      "incorrect argument to %s: \"%s\" : %s ",
@@ -1097,7 +1101,6 @@ public class SliderClient extends Abstra
   /**
    * Build an exit code for an application Id and its report.
    * If the report parameter is null, the app is killed
-   * @param appId app
    * @param report report
    * @return the exit code
    */
@@ -2071,8 +2074,6 @@ public class SliderClient extends Abstra
 
   /**
    * Look up an instance
-   * @param id instance ID
-   * @param serviceType service type
    * @return instance data
    * @throws UnknownApplicationInstanceException no match
    * @throws SliderException other failures

Added: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java?rev=1594045&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java (added)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java Mon May 12 18:30:30 2014
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public abstract class AbstractInputPropertiesValidator implements InputPropertiesValidator{
+
+  abstract void validatePropertyNamePrefix(String key) throws BadConfigException;
+
+  public void validate(ConfTreeOperations props)
+      throws BadConfigException {
+    validateGlobalProperties(props);
+    validateComponentProperties(props);
+
+  }
+
+  protected void validateComponentProperties(ConfTreeOperations props)
+      throws BadConfigException {
+    for (String compName : props.getComponentNames()) {
+      for (String key : props.getComponent(compName).keySet()) {
+        validatePropertyNamePrefix(key);
+      }
+    }
+  }
+
+  abstract void validateGlobalProperties(ConfTreeOperations props)
+      throws BadConfigException;
+
+}

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java Mon May 12 18:30:30 2014
@@ -64,15 +64,27 @@ public class ConfTreeOperations {
    * @throws BadConfigException
    */
   public void validate() throws BadConfigException {
+    validate(null);
+  }
+
+  /**
+   * Validate the configuration
+   * @param validator a provided properties validator
+   * @throws BadConfigException
+   */
+  public void validate(InputPropertiesValidator validator) throws BadConfigException {
     String version = confTree.schema;
     if (version == null) {
       throw new BadConfigException("'version' undefined");
     }
     if (!PersistKeys.SCHEMA.equals(version)) {
       throw new BadConfigException(
-        "version %s incompatible with supported version %s",
-        version,
-        PersistKeys.SCHEMA);
+          "version %s incompatible with supported version %s",
+          version,
+          PersistKeys.SCHEMA);
+    }
+    if (validator != null) {
+      validator.validate(this);
     }
   }
 
@@ -335,10 +347,21 @@ public class ConfTreeOperations {
    * @throws BadConfigException if the file is invalid
    */
   public void mergeFile(File file) throws IOException, BadConfigException {
+    mergeFile(file, null);
+  }
+
+  /**
+   * Load from a file and merge it in
+   * @param file file
+   * @param validator properties validator
+   * @throws IOException any IO problem
+   * @throws BadConfigException if the file is invalid
+   */
+  public void mergeFile(File file, InputPropertiesValidator validator) throws IOException, BadConfigException {
     ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
     ConfTree tree = confTreeSerDeser.fromFile(file);
     ConfTreeOperations ops = new ConfTreeOperations(tree);
-    ops.validate();
+    ops.validate(validator);
     merge(ops.confTree);
   }
 

Added: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java?rev=1594045&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java (added)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java Mon May 12 18:30:30 2014
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public interface InputPropertiesValidator {
+  void validate(ConfTreeOperations props) throws BadConfigException;
+}

Added: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java?rev=1594045&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java (added)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java Mon May 12 18:30:30 2014
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public class ResourcesInputPropertiesValidator
+    extends AbstractInputPropertiesValidator {
+
+  void validatePropertyNamePrefix(String key) throws BadConfigException {
+    if (!key.startsWith("yarn.")) {
+      throw new BadConfigException(
+          "argument %s does not have 'yarn.' prefix", key);
+    }
+  }
+
+  protected void validateGlobalProperties(ConfTreeOperations props)
+      throws BadConfigException {
+    for (String key : props.getGlobalOptions().keySet()) {
+      validatePropertyNamePrefix(key);
+    }
+  }
+}

Added: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java?rev=1594045&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java (added)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java Mon May 12 18:30:30 2014
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public class TemplateInputPropertiesValidator
+    extends AbstractInputPropertiesValidator {
+
+  void validatePropertyNamePrefix(String key) throws BadConfigException {
+    if (key.startsWith("yarn.")) {
+      throw new BadConfigException(
+          "argument %s does not have 'yarn.' prefix", key);
+    }
+  }
+
+  @Override
+  void validateGlobalProperties(ConfTreeOperations props) {
+    // do nothing
+  }
+}

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java Mon May 12 18:30:30 2014
@@ -31,7 +31,6 @@ import org.apache.slider.server.appmaste
 import org.apache.slider.server.appmaster.web.rest.AMWebServices;
 import org.apache.slider.server.appmaster.web.rest.SliderJacksonJaxbJsonProvider;
 import org.apache.slider.server.services.curator.CuratorHelper;
-import org.apache.slider.server.services.curator.RegistryBinderService;
 import org.apache.slider.server.services.registry.RegistryDiscoveryContext;
 import org.apache.slider.server.services.registry.RegistryRestResources;
 import org.apache.slider.server.services.registry.SliderRegistryService;

Modified: incubator/slider/trunk/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json (original)
+++ incubator/slider/trunk/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json Mon May 12 18:30:30 2014
@@ -10,7 +10,7 @@
 
   "components": {
     "slider-appmaster": {
-      "component.instances": "1",
+      "yarn.component.instances": "1",
       "yarn.vcores": "1",
       "yarn.memory": "256"
     }

Modified: incubator/slider/trunk/slider-core/src/test/app_packages/test_command_log/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/app_packages/test_command_log/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/app_packages/test_command_log/resources.json (original)
+++ incubator/slider/trunk/slider-core/src/test/app_packages/test_command_log/resources.json Mon May 12 18:30:30 2014
@@ -6,8 +6,8 @@
     },
     "components": {
         "COMMAND_LOGGER": {
-            "role.priority": "1",
-            "component.instances": "1"
+            "yarn.role.priority": "1",
+            "yarn.component.instances": "1"
         },
         "slider-appmaster": {
         }

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy Mon May 12 18:30:30 2014
@@ -44,7 +44,7 @@ public abstract class AgentMiniClusterTe
   public static void createSubConfFiles() {
     File destDir = new File("target/agent_minicluster_testbase")
     destDir.mkdirs()
-    agentConf = new File(destDir, "agentconf")
+    agentConf = new File(destDir, "agentconf.zip")
     agentConf.createNewFile()
     agentDef = new File(destDir, "agentdef")
     agentDef.createNewFile()

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeLoadExamples.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeLoadExamples.groovy?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeLoadExamples.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeLoadExamples.groovy Mon May 12 18:30:30 2014
@@ -18,6 +18,7 @@
 
 package org.apache.slider.core.conf
 
+import org.apache.slider.core.exceptions.BadConfigException
 import org.apache.slider.core.persist.JsonSerDeser
 import org.junit.Assert
 import org.junit.Test
@@ -53,4 +54,31 @@ class TestConfTreeLoadExamples extends A
     ops.validate()
 
   }
+
+  @Test
+  public void testLoadResourceWithValidator() throws Throwable {
+    def confTree = confTreeJsonSerDeser.fromResource(resource)
+    ConfTreeOperations ops = new ConfTreeOperations(confTree)
+    ops.resolve()
+    if (resource.endsWith("resources.json")) {
+      // these should pass since they are configured conrrectly with "yarn."
+      // properties
+      ops.validate(new ResourcesInputPropertiesValidator())
+    } else if (resource.startsWith("app_configuration")) {
+      ops.validate(new TemplateInputPropertiesValidator())
+    }
+    else {
+      // these have properties with other prefixes so they should generate
+      // BadConfigExceptions
+      try {
+        ops.validate(new ResourcesInputPropertiesValidator())
+        if ( !resource.endsWith(ExampleConfResources.empty)) {
+          fail (resource + " should have generated validation exception")
+        }
+      } catch (BadConfigException e) {
+         // ignore
+      }
+
+    }
+  }
 }

Modified: incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json (original)
+++ incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json Mon May 12 18:30:30 2014
@@ -12,14 +12,14 @@
   
   "components": {
     "master": {
-      "instances": "1",
+      "yarn.component.instances": "1",
       "yarn.vcores": "1",
       "yarn.memory": "1024"
     },
     "worker": {
-      "instances":"5",
+      "yarn.component.instances":"5",
       "yarn.vcores": "1",
       "yarn.memory": "512"
     }
   }
-}
\ No newline at end of file
+}

Modified: incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/providers/agent/tests/good/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/providers/agent/tests/good/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/providers/agent/tests/good/resources.json (original)
+++ incubator/slider/trunk/slider-core/src/test/resources/org/apache/slider/providers/agent/tests/good/resources.json Mon May 12 18:30:30 2014
@@ -6,8 +6,8 @@
   "components": {
     "node": {
       "yarn.memory": "256",
-      "component.instances": "5",
-      "role.priority":"1"
+      "yarn.component.instances": "5",
+      "yarn.role.priority":"1"
     }
   }
 }

Modified: incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/resources/org/apache/slider/providers/accumulo/instance/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/resources/org/apache/slider/providers/accumulo/instance/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/resources/org/apache/slider/providers/accumulo/instance/resources.json (original)
+++ incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/resources/org/apache/slider/providers/accumulo/instance/resources.json Mon May 12 18:30:30 2014
@@ -12,19 +12,19 @@
 
   "components": {
     "master": {
-      "component.instances": "1"
+      "yarn.component.instances": "1"
     },
       "tserver": {
-        "component.instances": "2"
+        "yarn.component.instances": "2"
       },
       "monitor": {
-        "component.instances": "0"
+        "yarn.component.instances": "0"
       },
       "tracer": {
-        "component.instances": "0"
+        "yarn.component.instances": "0"
       },
       "gc": {
-        "component.instances": "0"
+        "yarn.component.instances": "0"
       }
   }
 }
\ No newline at end of file

Modified: incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/resources/org/apache/slider/providers/hbase/instance/resources.json
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/resources/org/apache/slider/providers/hbase/instance/resources.json?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/resources/org/apache/slider/providers/hbase/instance/resources.json (original)
+++ incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/resources/org/apache/slider/providers/hbase/instance/resources.json Mon May 12 18:30:30 2014
@@ -11,12 +11,12 @@
 
   "components": {
     "master": {
-      "component.instances": "1",
+      "yarn.component.instances": "1",
       "yarn.vcores": "1",
       "yarn.memory": "1024"
     },
     "worker": {
-      "component.instances": "2",
+      "yarn.component.instances": "2",
       "yarn.vcores": "1",
       "yarn.memory": "768"
     }

Modified: incubator/slider/trunk/src/site/markdown/getting_started.md
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/src/site/markdown/getting_started.md?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/src/site/markdown/getting_started.md (original)
+++ incubator/slider/trunk/src/site/markdown/getting_started.md Mon May 12 18:30:30 2014
@@ -377,25 +377,24 @@ Once started, applications can be frozen
         "slider-appmaster" : {
         },
         "NIMBUS" : {
-            "role.priority" : "1",
-            "component.instances" : "1"
+            "yarn.role.priority" : "1",
+            "yarn.component.instances" : "1"
         },
         "STORM_REST_API" : {
-            "wait.heartbeat" : "3",
-            "role.priority" : "2",
-            "component.instances" : "1"
+            "yarn.role.priority" : "2",
+            "yarn.component.instances" : "1"
         },
         "STORM_UI_SERVER" : {
-            "role.priority" : "3",
-            "component.instances" : "1"
+            "yarn.role.priority" : "3",
+            "yarn.component.instances" : "1"
         },
         "DRPC_SERVER" : {
-            "role.priority" : "4",
-            "component.instances" : "1"
+            "yarn.role.priority" : "4",
+            "yarn.component.instances" : "1"
         },
         "SUPERVISOR" : {
-            "role.priority" : "5",
-            "component.instances" : "1"
+            "yarn.role.priority" : "5",
+            "yarn.component.instances" : "1"
         }
       }
     }
@@ -435,19 +434,14 @@ Once started, applications can be frozen
       },
       "components" : {
         "HBASE_MASTER" : {
-            "wait.heartbeat" : "5",
-            "role.priority" : "1",
-            "component.instances" : "1",
-            "role.script" : "scripts/hbase_master.py"
+            "yarn.role.priority" : "1",
+            "yarn.component.instances" : "1"
         },
         "slider-appmaster" : {
-            "jvm.heapsize" : "256M"
         },
         "HBASE_REGIONSERVER" : {
-            "wait.heartbeat" : "3",
-            "role.priority" : "2",
-            "component.instances" : "1",
-            "role.script" : "scripts/hbase_regionserver.py"
+            "yarn.role.priority" : "2",
+            "yarn.component.instances" : "1"
         }
       }
     }

Modified: incubator/slider/trunk/src/site/markdown/slider_specs/creating_app_definitions.md
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/src/site/markdown/slider_specs/creating_app_definitions.md?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/src/site/markdown/slider_specs/creating_app_definitions.md (original)
+++ incubator/slider/trunk/src/site/markdown/slider_specs/creating_app_definitions.md Mon May 12 18:30:30 2014
@@ -122,19 +122,15 @@ Sample:
       },
       "components" : {
           "HBASE_MASTER" : {
-              "wait.heartbeat" : "5",
-              "role.priority" : "1",
-              "component.instances" : "1",
-              "role.script" : "scripts/hbase_master.py"
+              "yarn.role.priority" : "1",
+              "yarn.component.instances" : "1"
           },
           "slider-appmaster" : {
               "jvm.heapsize" : "256M"
           },
           "HBASE_REGIONSERVER" : {
-              "wait.heartbeat" : "3",
-              "role.priority" : "2",
-              "component.instances" : "1",
-              "role.script" : "scripts/hbase_regionserver.py"
+              "yarn.role.priority" : "2",
+              "yarn.component.instances" : "1"
           }
       }
     }

Modified: incubator/slider/trunk/src/site/markdown/slider_specs/resource_specification.md
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/src/site/markdown/slider_specs/resource_specification.md?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/src/site/markdown/slider_specs/resource_specification.md (original)
+++ incubator/slider/trunk/src/site/markdown/slider_specs/resource_specification.md Mon May 12 18:30:30 2014
@@ -31,23 +31,20 @@ An example resource requirement for an a
     "components" : {
       "worker" : {
         "yarn.memory" : "768",
-        "env.MALLOC_ARENA_MAX" : "4",
-        "component.instances" : "1",
-        "component.name" : "worker",
+        "yarn.component.instances" : "1",
+        "yarn.component.name" : "worker",
         "yarn.vcores" : "1"
       },
       "slider" : {
         "yarn.memory" : "256",
-        "env.MALLOC_ARENA_MAX" : "4",
-        "component.instances" : "1",
-        "component.name" : "slider",
+        "yarn.component.instances" : "1",
+        "yarn.component.name" : "slider",
         "yarn.vcores" : "1"
       },
       "master" : {
         "yarn.memory" : "1024",
-        "env.MALLOC_ARENA_MAX" : "4",
-        "component.instances" : "1",
-        "component.name" : "master",
+        "yarn.component.instances" : "1",
+        "yarn.component.name" : "master",
         "yarn.vcores" : "1"
       }
     }

Modified: incubator/slider/trunk/src/site/markdown/specification/cli-actions.md
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/src/site/markdown/specification/cli-actions.md?rev=1594045&r1=1594044&r2=1594045&view=diff
==============================================================================
--- incubator/slider/trunk/src/site/markdown/specification/cli-actions.md (original)
+++ incubator/slider/trunk/src/site/markdown/specification/cli-actions.md Mon May 12 18:30:30 2014
@@ -350,7 +350,7 @@ which release requests have been made mu
 If the internal state of the Slider AM is defined as `AppState`
 
     forall r in clusterspec.roles :
-        r["component.instances"] ==
+        r["yarn.component.instances"] ==
           AppState.Roles[r].live + AppState.Roles[r].requested - AppState.Roles[r].released
 
 The `AppState` represents Slider's view of the external YARN system state, based on its
@@ -360,7 +360,7 @@ It is indirectly observable from the clu
 
 
     forall r in AM.getJSONClusterStatus().roles :
-        r["component.instances"] ==
+        r["yarn.component.instances"] ==
           r["role.actual.instances"] + r["role.requested.instances"] - r["role.releasing.instances"]
 
 Slider does not consider it an error if the number of actual instances remains below
@@ -473,7 +473,7 @@ which will change the desired steady-sta
     
     let updatedSpec = originalspec where:
         forall (name, size) in components :
-            updatedSpec.roles[name]["component.instances"] == size
+            updatedSpec.roles[name]["yarn.component.instances"] == size
     data(HDFS', cluster-json-path(HDFS', instancename)) == updatedSpec
     rpc-connection(slider-live-instances(YARN(t2))[0], SliderClusterProtocol)
     let flexed = rpc-connection(slider-live-instances(YARN(t2))[0], SliderClusterProtocol).flexClusterupdatedSpec)
@@ -496,7 +496,7 @@ and be `True` iff the desired steady sta
 #### Postconditions
 
     forall role in AppState.Roles.keys:
-        AppState'.Roles'[role].desiredCount = updatedSpec[roles]["component.instances"]
+        AppState'.Roles'[role].desiredCount = updatedSpec[roles]["yarn.component.instances"]
     result = AppState' != AppState