You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2015/10/19 12:16:08 UTC

[1/2] incubator-slider git commit: SLIDER-947 still trying to get node update events from YARN RM

Repository: incubator-slider
Updated Branches:
  refs/heads/feature/SLIDER-82-anti-affinity-attempt-2 3aeab9ca3 -> 88a7b34ca


SLIDER-947 still trying to get node update events from YARN RM


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

Branch: refs/heads/feature/SLIDER-82-anti-affinity-attempt-2
Commit: e2d8ab0daab639bae1421270bbd9ae2e7ec5810e
Parents: 3aeab9c
Author: Steve Loughran <st...@apache.org>
Authored: Sun Oct 18 19:22:09 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Sun Oct 18 19:22:09 2015 +0100

----------------------------------------------------------------------
 .../server/appmaster/management/BoolMetric.java |  3 +-
 .../agent/rest/LowLevelRestTestDelegates.groovy | 15 ++---
 .../slider/agent/rest/TestStandaloneREST.groovy | 19 +++---
 .../apache/slider/test/SliderTestUtils.groovy   | 64 +++++++++++++++++++-
 .../funtest/lifecycle/AgentRegistryIT.groovy    |  2 +-
 5 files changed, 75 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e2d8ab0d/slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java
index 23ea61d..33f8d85 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java
@@ -18,7 +18,6 @@
 
 package org.apache.slider.server.appmaster.management;
 
-import com.codahale.metrics.Counting;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Metric;
 
@@ -55,7 +54,7 @@ public class BoolMetric implements Metric, Gauge<Integer> {
    * @return true if the input parses to an integer other than 0. False if it doesn't parse
    * or parses to 0.
    */
-  public boolean fromString(String s) {
+  public static boolean fromString(String s) {
     try {
       return Integer.valueOf(s) != 0;
     } catch (NumberFormatException e) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e2d8ab0d/slider-core/src/test/groovy/org/apache/slider/agent/rest/LowLevelRestTestDelegates.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/LowLevelRestTestDelegates.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/LowLevelRestTestDelegates.groovy
index 3d87c28..ed10479 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/LowLevelRestTestDelegates.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/LowLevelRestTestDelegates.groovy
@@ -71,8 +71,8 @@ class LowLevelRestTestDelegates extends AbstractRestTestDelegate {
   
   public void logCodahaleMetrics() {
     // query Coda Hale metrics
-    log.info getWebPage(appmaster, SYSTEM_HEALTHCHECK)
-    log.info getWebPage(appmaster, SYSTEM_METRICS)
+    log.info prettyPrintJson(getWebPage(appmaster, SYSTEM_HEALTHCHECK))
+    log.info prettyPrintJson(getWebPage(appmaster, SYSTEM_METRICS))
   }
 
 
@@ -150,9 +150,6 @@ class LowLevelRestTestDelegates extends AbstractRestTestDelegate {
     assert components.size() >= 1
     log.info "${components}"
 
-    ComponentInformation amComponentInfo =
-        (ComponentInformation) components[COMPONENT_AM]
-
     ComponentInformation amFullInfo = fetchType(
         ComponentInformation,
         appmaster,
@@ -278,7 +275,6 @@ class LowLevelRestTestDelegates extends AbstractRestTestDelegate {
         MediaType.TEXT_PLAIN)
     log.info "Stopped: $outcome"
 
-    
     // now a ping is expected to fail
     String ping = appendToURL(appmaster, SLIDER_PATH_APPLICATION, ACTION_PING)
 
@@ -317,7 +313,6 @@ class LowLevelRestTestDelegates extends AbstractRestTestDelegate {
 
   @Override
   public void testSuiteGetOperations() {
-
     testCodahaleOperations()
     testMimeTypes()
     testLiveResources()
@@ -369,10 +364,8 @@ class LowLevelRestTestDelegates extends AbstractRestTestDelegate {
    * @return the outcome
    */
   Outcome probeForResolveConfValues(Map args) {
-    assert args["key"]
-    assert args["val"]
-    String key = args["key"]
-    String val = args["val"]
+    String key = requiredMapValue(args, "key")
+    String val = requiredMapValue(args, "val")
     ConfTreeOperations resolved = modelDesiredResolvedResources
 
     return Outcome.fromBool(resolved.get(key) == val)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e2d8ab0d/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
index 880d9ca..9808688 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
@@ -75,8 +75,8 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     }
     
     execOperation(WEB_STARTUP_TIME) {
-      def metrics = GET(directAM, SYSTEM_METRICS)
-      log.info metrics
+      def metrics = GET(directAM, SYSTEM_METRICS_JSON)
+      log.info prettyPrintJson(metrics)
     }
 
     GET(proxyAM)
@@ -87,15 +87,12 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     log.info GET(proxyAM, SYSTEM_METRICS_JSON)
 
     // using the metrics, await the first node status update
-    execOperation(WEB_STARTUP_TIME) {
-      def metrics = getMetrics(proxyAM)
-      if (!getGaugeAsBool(metrics,
-            "org.apache.slider.server.appmaster.state.RoleHistory.nodes-updated.flag", false)) {
-        throw new IOException("Nodes not updated in $metrics")
-      } else {
-        "true"
-      };
-    }
+    awaitGaugeValue(
+        appendToURL(proxyAM, SYSTEM_METRICS_JSON),
+        "org.apache.slider.server.appmaster.state.RoleHistory.nodes-updated.flag",
+        1,
+        WEB_STARTUP_TIME  * 2, 500)
+
     // Is the back door required? If so, don't test complex verbs via the proxy
     def proxyComplexVerbs = !SliderXmlConfKeys.X_DEV_INSECURE_REQUIRED
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e2d8ab0d/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 c5808f2..5de6aba 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
@@ -100,7 +100,12 @@ class SliderTestUtils extends Assert {
     log.info("");
   }
 
-  public static String prettyPrint(String json) {
+  /**
+   * Convert a JSON string to something readable
+   * @param json
+   * @return a string for printing
+   */
+  public static String prettyPrintJson(String json) {
     JsonOutput.prettyPrint(json)
   }
 
@@ -370,7 +375,7 @@ class SliderTestUtils extends Assert {
       if (timedOut) {
         duration.finish();
         describe("$operation: role count not met after $duration: $details")
-        log.info(prettyPrint(status.toJsonString()))
+        log.info(prettyPrintJson(status.toJsonString()))
         fail("$operation: role counts not met after $duration: " +
              details.toString() +
              " in \n$status ")
@@ -416,7 +421,7 @@ class SliderTestUtils extends Assert {
       String text,
       ClusterDescription status) {
     describe(text)
-    log.info(prettyPrint(status.toJsonString()))
+    log.info(prettyPrintJson(status.toJsonString()))
   }
 
 
@@ -1316,6 +1321,17 @@ class SliderTestUtils extends Assert {
   }
 
   /**
+   * Get a value from a map; raise an assertion if it is not there
+   * @param map map to look up
+   * @param key key
+   * @return the string value
+   */
+  String requiredMapValue(Map map, String key) {
+    assert map[key] != null
+    map[key].toString()
+  }
+
+  /**
    * Get a web page and deserialize the supplied JSON into
    * an instance of the specific class.
    * @param clazz class to deserialize to
@@ -1420,5 +1436,47 @@ class SliderTestUtils extends Assert {
     return metrics;
   }
 
+  /**
+   * Await a specific gauge being of the desired value
+   * @param target target URL
+   * @param gauge gauge name
+   * @param desiredValue desired value
+   * @param timeout timeout in millis
+   * @param sleepDur sleep in millis
+   */
+  public void awaitGaugeValue(String target, String gauge, int desiredValue,
+      int timeout,
+      int sleepDur) {
+    def text = "Probe $target for gauge $gauge == $desiredValue"
+    repeatUntilSuccess(text,
+      this.&probeMetricGaugeValue,
+      timeout, sleepDur,
+      [
+          url : target,
+          gauge: gauge,
+          desiredValue: desiredValue.toString()
+      ],
+      true, text) {
+       log.error(prettyPrintJson(GET(target)))
+    }
+  }
+
+  Outcome probeMetricGaugeValue(Map args) {
+    String url = requiredMapValue(args, "url")
+    String gauge = requiredMapValue(args, "gauge")
+    String vstr = requiredMapValue(args, "desiredValue")
+    assert vstr != null, "null desired value in $args"
+    assert vstr != "", "empty desired value in $args"
+    int desiredValue = Integer.decode(vstr)
+    try {
+      def metrics = parseMetrics(GET(url))
+      def gaugeValue = getGaugeValue(metrics, gauge, -1)
+      return gaugeValue == desiredValue ? Outcome.Success : Outcome.Retry
+    } catch (IOException e) {
+      return Outcome.Fail
+    }
+  }
+
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e2d8ab0d/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
index 4f9701c..ff5e57e 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
@@ -160,7 +160,7 @@ public class AgentRegistryIT extends AgentCommandTestBase
 
 
   Outcome probeForEntryMissing(Map args) {
-    String path = args["path"]
+    String path = requiredMapValue(args, "path")
     def shell = slider([ACTION_RESOLVE, ARG_PATH, path])
     return Outcome.fromBool(shell.ret == EXIT_NOT_FOUND)
   }


[2/2] incubator-slider git commit: SLIDER-82 IDE suggested AM cleanup

Posted by st...@apache.org.
SLIDER-82 IDE suggested AM cleanup


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

Branch: refs/heads/feature/SLIDER-82-anti-affinity-attempt-2
Commit: 88a7b34cacbbe7b092b7c02dbe1e653772fc441e
Parents: e2d8ab0
Author: Steve Loughran <st...@apache.org>
Authored: Mon Oct 19 11:09:10 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Oct 19 11:15:53 2015 +0100

----------------------------------------------------------------------
 .../server/appmaster/SliderAppMaster.java       | 35 ++++++--------------
 .../apache/slider/test/SliderTestUtils.groovy   | 10 +++---
 2 files changed, 17 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/88a7b34c/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 92b602f..777fa04 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -574,11 +574,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     String action = serviceArgs.getAction();
     List<String> actionArgs = serviceArgs.getActionArgs();
     int exitCode;
-/*  JDK7
-  switch (action) {
+    switch (action) {
       case SliderActions.ACTION_HELP:
-        log.info(getName() + serviceArgs.usage());
-        exitCode = LauncherExitCodes.EXIT_USAGE;
+        log.info("{}: {}", getName(), serviceArgs.usage());
+        exitCode = SliderExitCodes.EXIT_USAGE;
         break;
       case SliderActions.ACTION_CREATE:
         exitCode = createAndRunCluster(actionArgs.get(0));
@@ -586,20 +585,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       default:
         throw new SliderException("Unimplemented: " + action);
     }
-    */
-    if (action.equals(SliderActions.ACTION_HELP)) {
-      log.info("{}: {}", getName(), serviceArgs.usage());
-      exitCode = SliderExitCodes.EXIT_USAGE;
-    } else if (action.equals(SliderActions.ACTION_CREATE)) {
-      exitCode = createAndRunCluster(actionArgs.get(0));
-    } else {
-      throw new SliderException("Unimplemented: " + action);
-    }
     log.info("Exiting AM; final exit code = {}", exitCode);
     return exitCode;
   }
 
-
   /**
    * Initialize a newly created service then add it. 
    * Because the service is not started, this MUST be done before
@@ -705,23 +694,22 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     Map<String, String> envVars;
     List<Container> liveContainers;
-    
-    /**
+
+    /*
      * It is critical this section is synchronized, to stop async AM events
      * arriving while registering a restarting AM.
      */
     synchronized (appState) {
       int heartbeatInterval = HEARTBEAT_INTERVAL;
 
-      //add the RM client -this brings the callbacks in
-      asyncRMClient = AMRMClientAsync.createAMRMClientAsync(heartbeatInterval,
-                                                            this);
+      // add the RM client -this brings the callbacks in
+      asyncRMClient = AMRMClientAsync.createAMRMClientAsync(heartbeatInterval, this);
       addService(asyncRMClient);
       //now bring it up
       deployChildService(asyncRMClient);
 
 
-      //nmclient relays callbacks back to this class
+      // nmclient relays callbacks back to this class
       nmClientAsync = new NMClientAsyncImpl("nmclient", this);
       deployChildService(nmClientAsync);
 
@@ -756,8 +744,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       log.info(registryOperations.toString());
 
       //build the role map
-      List<ProviderRole> providerRoles =
-        new ArrayList<ProviderRole>(providerService.getRoles());
+      List<ProviderRole> providerRoles = new ArrayList<>(providerService.getRoles());
       providerRoles.addAll(SliderAMClientProvider.ROLES);
 
       // Start up the WebApp and track the URL for it
@@ -1020,7 +1007,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // up to date token for container launches (getContainerCredentials()).
     UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
     Credentials credentials = currentUser.getCredentials();
-    List<Text> filteredTokens = new ArrayList<Text>();
+    List<Text> filteredTokens = new ArrayList<>();
     filteredTokens.add(AMRMTokenIdentifier.KIND_NAME);
 
     boolean keytabProvided = securityConfiguration.isKeytabProvided();
@@ -1942,7 +1929,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   protected synchronized void launchProviderService(AggregateConf instanceDefinition,
                                                     File confDir)
     throws IOException, SliderException {
-    Map<String, String> env = new HashMap<String, String>();
+    Map<String, String> env = new HashMap<>();
     boolean execStarted = providerService.exec(instanceDefinition, confDir, env,
         this);
     if (execStarted) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/88a7b34c/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 5de6aba..ae07187 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
@@ -683,14 +683,18 @@ class SliderTestUtils extends Assert {
    *   to have been called.
    *   
    * @param path path to page
-   * @param connectionChecks optional closure to run against an open connection
    * @return body of response
    */
-  public static String getWebPage(String path, Closure connectionChecks = null) {
+  public static String getWebPage(String path) {
     HttpOperationResponse outcome = executeGet(path)
     return new String(outcome.data);
   }
 
+  /**
+   * Execute a GET operation
+   * @param path path to GET
+   * @return the response
+   */
   public static HttpOperationResponse executeGet(String path) {
     assert path
     assertHttpSupportInitialized()
@@ -1477,6 +1481,4 @@ class SliderTestUtils extends Assert {
     }
   }
 
-
-
 }