You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@twill.apache.org by ch...@apache.org on 2017/04/08 18:08:24 UTC

[01/24] twill git commit: (TWILL-186) Guard against YARN returning mismatch container size case.

Repository: twill
Updated Branches:
  refs/heads/site 7bdf857c0 -> 3680e00f0


(TWILL-186) Guard against YARN returning mismatch container size case.

- Also make sure we don't remove container request without adding it first
- Code cleanup for ApplicationMasterService and related classes
  - Get rid of the inner loop in the doRun method
    - The inner loop can block the heartbeat thread for too long if there are a lot of runnable instances to stop
  - Remove unnecessary throwables.propagate
  - Remove unnecessary intermediate method
  - Better logging
  - Request multiple instances in the same request
  - Refactory/simiply placement policy related code
  - Expose container instanceId instead of parsing it from runId

This closes #34 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: f4df32da26cbc7e4fb93f88fb0b1306575a9cf0e
Parents: 140f7da
Author: Terence Yim <ch...@apache.org>
Authored: Mon Feb 27 21:22:32 2017 -0800
Committer: Terence Yim <ch...@apache.org>
Committed: Wed Mar 1 16:18:51 2017 -0800

----------------------------------------------------------------------
 .../internal/TwillContainerController.java      |  10 +-
 .../twill/internal/TwillContainerLauncher.java  |   5 +
 .../appmaster/ApplicationMasterService.java     | 198 ++++++++-----------
 .../internal/appmaster/ExpectedContainers.java  |  13 +-
 .../appmaster/PlacementPolicyManager.java       |  77 ++------
 .../appmaster/RunnableContainerRequest.java     |   6 +-
 .../internal/appmaster/RunningContainers.java   |  11 +-
 .../internal/yarn/AbstractYarnAMClient.java     |  50 ++---
 .../twill/internal/yarn/YarnAMClient.java       |  40 +++-
 9 files changed, 186 insertions(+), 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
index 692e6b2..15689f5 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
@@ -22,6 +22,8 @@ import com.google.common.util.concurrent.Service;
 import org.apache.twill.api.ServiceController;
 import org.apache.twill.internal.state.Message;
 
+import javax.annotation.Nullable;
+
 /**
  * A {@link ServiceController} that allows sending a message directly. Internal use only.
  */
@@ -36,7 +38,13 @@ public interface TwillContainerController extends ServiceController, Service {
   void completed(int exitStatus);
 
   /**
-   * @returns the container's live node data.
+   * @return the container's live node data.
    */
+  @Nullable
   ContainerLiveNodeData getLiveNodeData();
+
+  /**
+   * @return the instance ID of the runnable that running in the container controlled by this controller.
+   */
+  int getInstanceId();
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
index 8dce91e..9b6384c 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
@@ -262,6 +262,11 @@ public final class TwillContainerLauncher {
       processController.cancel();
     }
 
+    @Override
+    public int getInstanceId() {
+      return instanceId;
+    }
+
     private void killAndWait(int maxWaitSecs) {
       Stopwatch watch = new Stopwatch();
       watch.start();

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index d5beb69..b4ac288 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -21,7 +21,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
 import com.google.common.base.Supplier;
-import com.google.common.base.Throwables;
 import com.google.common.collect.DiscreteDomains;
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.ImmutableSet;
@@ -160,8 +159,8 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
                                                         Long.parseLong(System.getenv(EnvKeys.YARN_APP_ID_CLUSTER_TIME)),
                                                         amClient.getContainerId().toString(), getLocalizeFiles());
 
-    this.expectedContainers = initExpectedContainers(twillSpec);
-    this.runningContainers = initRunningContainers(amClient.getContainerId(), amClient.getHost());
+    this.expectedContainers = new ExpectedContainers(twillSpec);
+    this.runningContainers = createRunningContainers(amClient.getContainerId(), amClient.getHost());
     this.eventHandler = createEventHandler(twillSpec);
   }
 
@@ -180,26 +179,22 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
 
   @SuppressWarnings("unchecked")
   @Nullable
-  private EventHandler createEventHandler(TwillSpecification twillSpec) {
-    try {
-      // Should be able to load by this class ClassLoader, as they packaged in the same jar.
-      EventHandlerSpecification handlerSpec = twillSpec.getEventHandler();
-      if (handlerSpec == null) {
-        return null;
-      }
-
-      Class<?> handlerClass = getClass().getClassLoader().loadClass(handlerSpec.getClassName());
-      Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
-                                  "Class {} does not implements {}",
-                                  handlerClass, EventHandler.class.getName());
-      return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
-    } catch (Exception e) {
-      throw Throwables.propagate(e);
+  private EventHandler createEventHandler(TwillSpecification twillSpec) throws ClassNotFoundException {
+    // Should be able to load by this class ClassLoader, as they packaged in the same jar.
+    EventHandlerSpecification handlerSpec = twillSpec.getEventHandler();
+    if (handlerSpec == null) {
+      return null;
     }
+
+    Class<?> handlerClass = getClass().getClassLoader().loadClass(handlerSpec.getClassName());
+    Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
+                                "Class {} does not implements {}",
+                                handlerClass, EventHandler.class.getName());
+    return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
   }
 
-  private RunningContainers initRunningContainers(ContainerId appMasterContainerId,
-                                                  String appMasterHost) throws Exception {
+  private RunningContainers createRunningContainers(ContainerId appMasterContainerId,
+                                                    String appMasterHost) throws Exception {
     TwillRunResources appMasterResources = new DefaultTwillRunResources(
       0,
       appMasterContainerId.toString(),
@@ -211,14 +206,6 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
       twillSpec.getRunnables(), twillRuntimeSpec.getMaxRetries());
   }
 
-  private ExpectedContainers initExpectedContainers(TwillSpecification twillSpec) {
-    Map<String, Integer> expectedCounts = Maps.newHashMap();
-    for (RuntimeSpecification runtimeSpec : twillSpec.getRunnables().values()) {
-      expectedCounts.put(runtimeSpec.getName(), runtimeSpec.getResourceSpecification().getInstances());
-    }
-    return new ExpectedContainers(expectedCounts);
-  }
-
   @Override
   public ResourceReport get() {
     return runningContainers.getResourceReport();
@@ -392,8 +379,14 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     boolean isRequestRelaxed = false;
     long nextTimeoutCheck = System.currentTimeMillis() + Constants.PROVISION_TIMEOUT;
     while (!stopped) {
-      // Call allocate. It has to be made at first in order to be able to get cluster resource availability.
-      amClient.allocate(0.0f, allocateHandler);
+      TimeUnit.SECONDS.sleep(1);
+
+      try {
+        // Call allocate. It has to be made at first in order to be able to get cluster resource availability.
+        amClient.allocate(0.0f, allocateHandler);
+      } catch (Exception e) {
+        LOG.warn("Exception raised when making heartbeat to RM. Will be retried in next heartbeat.", e);
+      }
 
       // Looks for containers requests.
       if (provisioning.isEmpty() && runnableContainerRequests.isEmpty() && runningContainers.isEmpty()) {
@@ -402,28 +395,22 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
       }
 
       // If nothing is in provisioning, and no pending request, move to next one
-      int count = runnableContainerRequests.size();
-      LOG.debug("Runnable container requests: {}", count);
-      while (provisioning.isEmpty() && currentRequest == null && !runnableContainerRequests.isEmpty()) {
-        RunnableContainerRequest runnableContainerRequest = runnableContainerRequests.peek();
-        if (!runnableContainerRequest.isReadyToBeProvisioned()) {
-          // take it out from queue and put it back at the end for second chance.
-          runnableContainerRequest = runnableContainerRequests.poll();
-          runnableContainerRequests.add(runnableContainerRequest);
-          LOG.debug("Request not ready: {}", runnableContainerRequest);
-
-          // We checked all the requests that were pending when we started this loop
-          // Any remaining requests are not ready to be provisioned
-          if (--count <= 0) {
-            break;
-          }
+      if (provisioning.isEmpty() && currentRequest == null && !runnableContainerRequests.isEmpty()) {
+        RunnableContainerRequest containerRequest = runnableContainerRequests.peek();
+        // If the request at the head of the request queue is not yet ready, move it to the end of the queue
+        // so that it won't block requests that are already ready
+        if (!containerRequest.isReadyToBeProvisioned()) {
+          LOG.debug("Request not ready: {}", containerRequest);
+          runnableContainerRequests.add(runnableContainerRequests.poll());
           continue;
         }
-        currentRequest = runnableContainerRequest.takeRequest();
+
+        currentRequest = containerRequest.takeRequest();
         if (currentRequest == null) {
           // All different types of resource request from current order is done, move to next one
           // TODO: Need to handle order type as well
           runnableContainerRequests.poll();
+          continue;
         }
       }
 
@@ -448,10 +435,6 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
       }
 
       nextTimeoutCheck = checkProvisionTimeout(nextTimeoutCheck);
-
-      if (isRunning()) {
-        TimeUnit.SECONDS.sleep(1);
-      }
     }
   }
 
@@ -462,28 +445,27 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     amClient.clearBlacklist();
 
     //Check the allocation strategy
-    AllocationSpecification currentAllocationSpecification = request.getKey();
-    if (currentAllocationSpecification.getType().equals(AllocationSpecification.Type.ALLOCATE_ONE_INSTANCE_AT_A_TIME)) {
-
-      //Check the placement policy
-      TwillSpecification.PlacementPolicy placementPolicy =
-        placementPolicyManager.getPlacementPolicy(currentAllocationSpecification.getRunnableName());
-      if (placementPolicy != null
-        && placementPolicy.getType().equals(TwillSpecification.PlacementPolicy.Type.DISTRIBUTED)) {
-
-        //Update blacklist with hosts which are running DISTRIBUTED runnables
-        for (String runnable : placementPolicyManager.getFellowRunnables(request.getKey().getRunnableName())) {
-          Collection<ContainerInfo> containerStats =
-            runningContainers.getContainerInfo(runnable);
-          for (ContainerInfo containerInfo : containerStats) {
-            // Yarn Resource Manager may include port in the node name depending on the setting
-            // YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME. It is safe to add both
-            // the names (with and without port) to the blacklist.
-            LOG.debug("Adding {} to host blacklist", containerInfo.getHost().getHostName());
-            amClient.addToBlacklist(containerInfo.getHost().getHostName());
-            amClient.addToBlacklist(containerInfo.getHost().getHostName() + ":" + containerInfo.getPort());
-          }
-        }
+    AllocationSpecification allocationSpec = request.getKey();
+    if (!allocationSpec.getType().equals(AllocationSpecification.Type.ALLOCATE_ONE_INSTANCE_AT_A_TIME)) {
+      return;
+    }
+
+    //Check the placement policy
+    String runnableName = allocationSpec.getRunnableName();
+    TwillSpecification.PlacementPolicy placementPolicy = placementPolicyManager.getPlacementPolicy(runnableName);
+    if (placementPolicy == null || placementPolicy.getType() != TwillSpecification.PlacementPolicy.Type.DISTRIBUTED) {
+      return;
+    }
+
+    //Update blacklist with hosts which are running DISTRIBUTED runnables
+    for (String runnable : placementPolicy.getNames()) {
+      for (ContainerInfo containerInfo : runningContainers.getContainerInfo(runnable)) {
+        // Yarn Resource Manager may include port in the node name depending on the setting
+        // YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME. It is safe to add both
+        // the names (with and without port) to the blacklist.
+        LOG.debug("Adding {} to host blacklist", containerInfo.getHost().getHostName());
+        amClient.addToBlacklist(containerInfo.getHost().getHostName());
+        amClient.addToBlacklist(containerInfo.getHost().getHostName() + ":" + containerInfo.getPort());
       }
     }
   }
@@ -501,9 +483,7 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
 
     for (Multiset.Entry<String> entry : restartRunnables.entrySet()) {
       LOG.info("Re-request container for {} with {} instances.", entry.getElement(), entry.getCount());
-      for (int i = 0; i < entry.getCount(); i++) {
-        runnableContainerRequests.add(createRunnableContainerRequest(entry.getElement()));
-      }
+      runnableContainerRequests.add(createRunnableContainerRequest(entry.getElement(),  entry.getCount()));
     }
 
     // For all runnables that needs to re-request for containers, update the expected count timestamp
@@ -584,10 +564,10 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     Queue<RunnableContainerRequest> requests = new ConcurrentLinkedQueue<>();
     // For each order in the twillSpec, create container request for runnables, depending on Placement policy.
     for (TwillSpecification.Order order : twillSpec.getOrders()) {
-      Set<String> distributedRunnables = placementPolicyManager.getDistributedRunnables(order.getNames());
-      Set<String> defaultRunnables = Sets.newHashSet();
-      defaultRunnables.addAll(order.getNames());
-      defaultRunnables.removeAll(distributedRunnables);
+      Set<String> distributedRunnables = Sets.intersection(placementPolicyManager.getDistributedRunnables(),
+                                                           order.getNames());
+      Set<String> defaultRunnables = Sets.difference(order.getNames(), distributedRunnables);
+
       Map<AllocationSpecification, Collection<RuntimeSpecification>> requestsMap = Maps.newHashMap();
       for (String runnableName : distributedRunnables) {
         RuntimeSpecification runtimeSpec = twillSpec.getRunnables().get(runnableName);
@@ -637,19 +617,19 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
           //Spawning 1 instance at a time
           newContainers = 1;
         }
+
+        // TODO: Allow user to set priority?
+        LOG.info("Request {} containers with capability {} for runnable {}", newContainers, capability, name);
+        YarnAMClient.ContainerRequestBuilder builder = amClient.addContainerRequest(capability, newContainers);
+        builder.setPriority(0);
+
         TwillSpecification.PlacementPolicy placementPolicy = placementPolicyManager.getPlacementPolicy(name);
-        Set<String> hosts = Sets.newHashSet();
-        Set<String> racks = Sets.newHashSet();
         if (placementPolicy != null) {
-          hosts = placementPolicy.getHosts();
-          racks = placementPolicy.getRacks();
+          builder.addHosts(placementPolicy.getHosts())
+                 .addRacks(placementPolicy.getRacks());
         }
-        // TODO: Allow user to set priority?
-        LOG.info("Request {} containers with capability {} for runnable {}", newContainers, capability, name);
-        String requestId = amClient.addContainerRequest(capability, newContainers)
-          .addHosts(hosts)
-          .addRacks(racks)
-          .setPriority(0).apply();
+
+        String requestId = builder.apply();
         provisioning.add(new ProvisionRequest(runtimeSpec, requestId, newContainers, allocationType));
       }
     }
@@ -661,14 +641,14 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
   private void launchRunnable(List<? extends ProcessLauncher<YarnContainerInfo>> launchers,
                               Queue<ProvisionRequest> provisioning) {
     for (ProcessLauncher<YarnContainerInfo> processLauncher : launchers) {
-      LOG.info("Got container {}", processLauncher.getContainerInfo().getId());
+      LOG.info("Container allocated: {}", processLauncher.getContainerInfo().getContainer());
       ProvisionRequest provisionRequest = provisioning.peek();
       if (provisionRequest == null) {
         continue;
       }
 
       String runnableName = provisionRequest.getRuntimeSpec().getName();
-      LOG.info("Starting runnable {} with {}", runnableName, processLauncher);
+      LOG.info("Starting runnable {} in {}", runnableName, processLauncher.getContainerInfo().getContainer());
 
       LOG.debug("Log level for Twill runnable {} is {}", runnableName,
                 twillRuntimeSpec.getLogLevels().get(runnableName).get(Logger.ROOT_LOGGER_NAME));
@@ -713,17 +693,15 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     }
   }
 
-  private List<LocalFile> getLocalizeFiles() {
+  private List<LocalFile> getLocalizeFiles() throws IOException {
     try (Reader reader = Files.newBufferedReader(Paths.get(Constants.Files.LOCALIZE_FILES), StandardCharsets.UTF_8)) {
       return new GsonBuilder().registerTypeAdapter(LocalFile.class, new LocalFileCodec())
         .create().fromJson(reader, new TypeToken<List<LocalFile>>() {
         }.getType());
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
     }
   }
 
-  private Map<String, Map<String, String>> getEnvironments() {
+  private Map<String, Map<String, String>> getEnvironments() throws IOException {
     Path envFile = Paths.get(Constants.Files.RUNTIME_CONFIG_JAR, Constants.Files.ENVIRONMENTS);
     if (!Files.exists(envFile)) {
       return new HashMap<>();
@@ -732,8 +710,6 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     try (Reader reader = Files.newBufferedReader(envFile, StandardCharsets.UTF_8)) {
       return new Gson().fromJson(reader, new TypeToken<Map<String, Map<String, String>>>() {
       }.getType());
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
     }
   }
 
@@ -823,10 +799,6 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     };
   }
 
-  private RunnableContainerRequest createRunnableContainerRequest(final String runnableName) {
-    return createRunnableContainerRequest(runnableName, 1);
-  }
-
   private RunnableContainerRequest createRunnableContainerRequest(final String runnableName,
                                                                   final int numberOfInstances) {
     return createRunnableContainerRequest(runnableName, numberOfInstances, true);
@@ -846,8 +818,8 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     RuntimeSpecification runtimeSpec = twillSpec.getRunnables().get(runnableName);
     Resource capability = createCapability(runtimeSpec.getResourceSpecification());
     Map<AllocationSpecification, Collection<RuntimeSpecification>> requestsMap = Maps.newHashMap();
-    if (placementPolicyManager.getPlacementPolicyType(runnableName).equals(
-      TwillSpecification.PlacementPolicy.Type.DISTRIBUTED)) {
+
+    if (placementPolicyManager.getDistributedRunnables().contains(runnableName)) {
       for (int instanceId = 0; instanceId < numberOfInstances; instanceId++) {
         AllocationSpecification allocationSpecification =
           new AllocationSpecification(capability, AllocationSpecification.Type.ALLOCATE_ONE_INSTANCE_AT_A_TIME,
@@ -938,18 +910,9 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
   /**
    * Helper method to restart instances of runnables.
    */
-  private void restartRunnableInstances(String runnableName, @Nullable Set<Integer> instanceIds,
+  private void restartRunnableInstances(final String runnableName, @Nullable final Set<Integer> instanceIds,
                                         final Runnable completion) {
-    Runnable restartInstancesRunnable = createRestartInstancesRunnable(runnableName, instanceIds, completion);
-    instanceChangeExecutor.execute(restartInstancesRunnable);
-  }
-
-  /**
-   * Creates a Runnable for execution of restart instances request.
-   */
-  private Runnable createRestartInstancesRunnable(final String runnableName, @Nullable final Set<Integer> instanceIds,
-                                                  final Runnable completion) {
-    return new Runnable() {
+    instanceChangeExecutor.execute(new Runnable() {
       @Override
       public void run() {
         LOG.debug("Begin restart runnable {} instances.", runnableName);
@@ -974,8 +937,11 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
           }
         }
 
+        LOG.info("All instances in {} for runnable {} are stopped. Ready to provision",
+                 instancesToRemove, runnableName);
+
         // set the container request to be ready
-        containerRequest.setReadyToBeProvisioned(true);
+        containerRequest.setReadyToBeProvisioned();
 
         // For all runnables that needs to re-request for containers, update the expected count timestamp
         // so that the EventHandler would be triggered with the right expiration timestamp.
@@ -983,7 +949,7 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
 
         completion.run();
       }
-    };
+    });
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ExpectedContainers.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ExpectedContainers.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ExpectedContainers.java
index f4ebbd0..c0ffdb0 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ExpectedContainers.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ExpectedContainers.java
@@ -19,6 +19,8 @@ package org.apache.twill.internal.appmaster;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
+import org.apache.twill.api.RuntimeSpecification;
+import org.apache.twill.api.TwillSpecification;
 
 import java.util.Map;
 
@@ -30,13 +32,14 @@ final class ExpectedContainers {
 
   private final Map<String, ExpectedCount> expectedCounts;
 
-  ExpectedContainers(Map<String, Integer> expected) {
-    expectedCounts = Maps.newHashMap();
+  ExpectedContainers(TwillSpecification twillSpec) {
+    Map<String, ExpectedCount> expectedCounts = Maps.newHashMap();
     long now = System.currentTimeMillis();
-
-    for (Map.Entry<String, Integer> entry : expected.entrySet()) {
-      expectedCounts.put(entry.getKey(), new ExpectedCount(entry.getValue(), now));
+    for (RuntimeSpecification runtimeSpec : twillSpec.getRunnables().values()) {
+      expectedCounts.put(runtimeSpec.getName(),
+                         new ExpectedCount(runtimeSpec.getResourceSpecification().getInstances(), now));
     }
+    this.expectedCounts = expectedCounts;
   }
 
   synchronized void setExpected(String runnable, int expected) {

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/PlacementPolicyManager.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/PlacementPolicyManager.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/PlacementPolicyManager.java
index 91a58bb..a4e6af5 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/PlacementPolicyManager.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/PlacementPolicyManager.java
@@ -18,60 +18,42 @@
 
 package org.apache.twill.internal.appmaster;
 
-import com.google.common.collect.Sets;
 import org.apache.twill.api.TwillSpecification;
 
 import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
 
 /**
  * This class provides helper functions for operating on a set of Placement Policies.
  */
-public class PlacementPolicyManager {
-  List<TwillSpecification.PlacementPolicy> placementPolicies;
+final class PlacementPolicyManager {
 
-  public PlacementPolicyManager(List<TwillSpecification.PlacementPolicy> placementPolicies) {
-    this.placementPolicies = placementPolicies;
-  }
+  private final Map<TwillSpecification.PlacementPolicy.Type, Set<String>> policyTypeToRunnables;
+  private final Map<String, TwillSpecification.PlacementPolicy> runnablePolicies;
 
-  /**
-   * Given a set of runnables, get all runnables which belong to DISTRIBUTED placement policies.
-   * @param givenRunnables Set of runnables.
-   * @return Subset of runnables, which belong to DISTRIBUTED placement policies.
-   */
-  public Set<String> getDistributedRunnables(Set<String> givenRunnables) {
-    Set<String> distributedRunnables = getAllDistributedRunnables();
-    distributedRunnables.retainAll(givenRunnables);
-    return distributedRunnables;
-  }
+  PlacementPolicyManager(List<TwillSpecification.PlacementPolicy> policies) {
+    this.policyTypeToRunnables = new EnumMap<>(TwillSpecification.PlacementPolicy.Type.class);
+    this.runnablePolicies = new HashMap<>();
 
-  /**
-   * Given a runnable, get the type of placement policy. Returns DEFAULT if no placement policy is specified.
-   * @param runnableName Name of runnable.
-   * @return Placement policy type of the runnable.
-   */
-  public TwillSpecification.PlacementPolicy.Type getPlacementPolicyType(String runnableName) {
-    for (TwillSpecification.PlacementPolicy placementPolicy : placementPolicies) {
-      if (placementPolicy.getNames().contains(runnableName)) {
-        return placementPolicy.getType();
+    for (TwillSpecification.PlacementPolicy policy : policies) {
+      policyTypeToRunnables.put(policy.getType(), policy.getNames());
+      for (String runnable : policy.getNames()) {
+        runnablePolicies.put(runnable, policy);
       }
     }
-    return TwillSpecification.PlacementPolicy.Type.DEFAULT;
   }
 
   /**
-   * Get all runnables which belong to the same Placement policy as the given runnable.
-   * @param runnableName Name of runnable.
-   * @return Set of runnables, with same placement policy.
+   * Returns all runnables which belong to DISTRIBUTED placement policies.
    */
-  public Set<String> getFellowRunnables(String runnableName) {
-    for (TwillSpecification.PlacementPolicy placementPolicy : placementPolicies) {
-      if (placementPolicy.getNames().contains(runnableName)) {
-        return placementPolicy.getNames();
-      }
-    }
-    return Collections.emptySet();
+  Set<String> getDistributedRunnables() {
+    Set<String> runnables = policyTypeToRunnables.get(TwillSpecification.PlacementPolicy.Type.DISTRIBUTED);
+    return runnables == null ? Collections.<String>emptySet() : runnables;
   }
 
   /**
@@ -80,25 +62,8 @@ public class PlacementPolicyManager {
    * @param runnableName Name of runnable.
    * @return Placement policy of the runnable.
    */
-  public TwillSpecification.PlacementPolicy getPlacementPolicy(String runnableName) {
-    for (TwillSpecification.PlacementPolicy placementPolicy : placementPolicies) {
-      if (placementPolicy.getNames().contains(runnableName)) {
-        return placementPolicy;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Gets all runnables which belong to DISTRIBUTED placement policies.
-   */
-  private Set<String> getAllDistributedRunnables() {
-    Set<String> distributedRunnables = Sets.newHashSet();
-    for (TwillSpecification.PlacementPolicy placementPolicy : placementPolicies) {
-      if (placementPolicy.getType().equals(TwillSpecification.PlacementPolicy.Type.DISTRIBUTED)) {
-        distributedRunnables.addAll(placementPolicy.getNames());
-      }
-    }
-    return  distributedRunnables;
+  @Nullable
+  TwillSpecification.PlacementPolicy getPlacementPolicy(String runnableName) {
+    return runnablePolicies.get(runnableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
index e001121..1dcffed 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
@@ -53,12 +53,12 @@ final class RunnableContainerRequest {
     return orderType;
   }
 
-  public boolean isReadyToBeProvisioned() {
+  boolean isReadyToBeProvisioned() {
     return isReadyToBeProvisioned;
   }
 
-  public void setReadyToBeProvisioned(boolean isProvisioned) {
-    this.isReadyToBeProvisioned = isProvisioned;
+  void setReadyToBeProvisioned() {
+    this.isReadyToBeProvisioned = true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
index 0763f26..a950c46 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
@@ -236,7 +236,7 @@ final class RunningContainers {
     try {
       // Find the controller with particular instance id.
       for (Map.Entry<String, TwillContainerController> entry : containers.row(runnableName).entrySet()) {
-        if (getInstanceId(entry.getValue().getRunId()) == instanceId) {
+        if (entry.getValue().getInstanceId() == instanceId) {
           containerId = entry.getKey();
           controller = entry.getValue();
           break;
@@ -449,7 +449,7 @@ final class RunningContainers {
 
       for (Map.Entry<String, TwillContainerController> completedEntry : lookup.entrySet()) {
         TwillContainerController controller = completedEntry.getValue();
-        instanceId = getInstanceId(controller.getRunId());
+        instanceId = controller.getInstanceId();
 
         // TODO: Can there be multiple controllers for a single container?
         // TODO: What is the best way to determine whether to restart container when there are multiple controllers?
@@ -466,7 +466,7 @@ final class RunningContainers {
         }
         // TODO: should we remove the completed instance from instanceId and resource report even on failures?
         // TODO: won't they get added back when the container is re-requested?
-        removeInstanceId(runnableName, getInstanceId(controller.getRunId()));
+        removeInstanceId(runnableName, controller.getInstanceId());
         resourceReport.removeRunnableResources(runnableName, containerId);
       }
       
@@ -651,11 +651,6 @@ final class RunningContainers {
     return RunIds.fromString(baseId.getId() + '-' + instanceId);
   }
 
-  private int getInstanceId(RunId runId) {
-    String id = runId.getId();
-    return Integer.parseInt(id.substring(id.lastIndexOf('-') + 1));
-  }
-
   /**
    * Given the containerId, removes the corresponding containerInfo.
    * @param containerId Id for the container to be removed.

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java
index d28f810..e8f3b2b 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java
@@ -36,6 +36,7 @@ import org.slf4j.LoggerFactory;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import javax.annotation.Nullable;
@@ -50,12 +51,11 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
   private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnAMClient.class);
 
   // Map from a unique ID to inflight requests
-  private final Multimap<String, T> containerRequests;
-
-  // List of requests pending to send through allocate call
-  private final List<T> requests;
+  private final Multimap<String, T> inflightRequests;
+  // Map from a unique ID to pending requests that are not yet submitted to YARN
+  private final Multimap<String, T> pendingRequests;
   // List of requests pending to remove through allocate call
-  private final List<T> removes;
+  private final List<T> pendingRemoves;
   //List of pending blacklist additions for the next allocate call
   private final List<String> blacklistAdditions;
   //List of pending blacklist removals for the next allocate call
@@ -81,9 +81,9 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
     Preconditions.checkArgument(masterContainerId != null,
                                 "Missing %s from environment", containerIdEnvName);
     this.containerId = ConverterUtils.toContainerId(masterContainerId);
-    this.containerRequests = ArrayListMultimap.create();
-    this.requests = Lists.newLinkedList();
-    this.removes = Lists.newLinkedList();
+    this.inflightRequests = ArrayListMultimap.create();
+    this.pendingRequests = ArrayListMultimap.create();
+    this.pendingRemoves = Lists.newLinkedList();
     this.blacklistAdditions = Lists.newArrayList();
     this.blacklistRemovals = Lists.newArrayList();
     this.blacklistedResources = Lists.newArrayList();
@@ -109,16 +109,17 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
     // With bug YARN-314, if we mix the allocate call with new container request of the same priority,
     // in some cases the RM would not see the new request (based on sorting of resource capability),
     // but rather only see the one with size = 0.
-    if (removes.isEmpty()) {
-      for (T request : requests) {
-        addContainerRequest(request);
+    if (pendingRemoves.isEmpty()) {
+      for (Map.Entry<String, T> entry : pendingRequests.entries()) {
+        addContainerRequest(entry.getValue());
       }
-      requests.clear();
+      inflightRequests.putAll(pendingRequests);
+      pendingRequests.clear();
     } else {
-      for (T request : removes) {
+      for (T request : pendingRemoves) {
         removeContainerRequest(request);
       }
-      removes.clear();
+      pendingRemoves.clear();
     }
 
     if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
@@ -131,7 +132,12 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
     List<RunnableProcessLauncher> launchers = allocateResponse.getLaunchers();
 
     if (!launchers.isEmpty()) {
-      handler.acquired(launchers);
+      // Only call handler acquire if there is actually inflight requests.
+      // This is to workaround the YARN behavior that it can return more containers being asked,
+      // such that it causes us to launch process in the pending requests with the wrong container size
+      if (!inflightRequests.isEmpty()) {
+        handler.acquired(launchers);
+      }
 
       // If no process has been launched through the given launcher, return the container.
       for (ProcessLauncher<YarnContainerInfo> l : launchers) {
@@ -140,7 +146,7 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
         if (!launcher.isLaunched()) {
           YarnContainerInfo containerInfo = launcher.getContainerInfo();
           // Casting is needed in Java 8, otherwise it complains about ambiguous method over the info(String, Throwable)
-          LOG.info("Nothing to run in container, releasing it: {}", (Object) containerInfo.getContainer());
+          LOG.info("Nothing to run in container, releasing it: {}", containerInfo.getContainer());
           releaseAssignedContainer(containerInfo);
         }
       }
@@ -153,11 +159,6 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
   }
 
   @Override
-  public final ContainerRequestBuilder addContainerRequest(Resource capability) {
-    return addContainerRequest(capability, 1);
-  }
-
-  @Override
   public final ContainerRequestBuilder addContainerRequest(Resource capability, int count) {
     return new ContainerRequestBuilder(adjustCapability(capability), count) {
       @Override
@@ -170,8 +171,7 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
 
           for (int i = 0; i < count; i++) {
             T request = createContainerRequest(priority, capability, hosts, racks, relaxLocality);
-            containerRequests.put(id, request);
-            requests.add(request);
+            pendingRequests.put(id, request);
           }
 
           return id;
@@ -208,8 +208,8 @@ public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implem
 
   @Override
   public final synchronized void completeContainerRequest(String id) {
-    for (T request : containerRequests.removeAll(id)) {
-      removes.add(request);
+    for (T request : inflightRequests.removeAll(id)) {
+      pendingRemoves.add(request);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/twill/blob/f4df32da/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
index a10181e..65856ca 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
@@ -86,10 +86,19 @@ public interface YarnAMClient extends Service {
     }
   }
 
+  /**
+   * Returns the container ID of the application.
+   */
   ContainerId getContainerId();
 
+  /**
+   * Returns the hostname of the node manager that the AM is running on
+   */
   String getHost();
 
+  /**
+   * Returns the port of the node manager that the AM is running on
+   */
   int getNMPort();
 
   /**
@@ -98,19 +107,10 @@ public interface YarnAMClient extends Service {
   void setTracker(InetSocketAddress trackerAddr, URL trackerUrl);
 
   /**
-   * Callback for allocate call.
+   * The heartbeat call to RM.
    */
-  // TODO: Move AM heartbeat logic into this interface so AM only needs to handle callback.
-  interface AllocateHandler {
-    void acquired(List<? extends ProcessLauncher<YarnContainerInfo>> launchers);
-
-    void completed(List<YarnContainerStatus> completed);
-  }
-
   void allocate(float progress, AllocateHandler handler) throws Exception;
 
-  ContainerRequestBuilder addContainerRequest(Resource capability);
-
   ContainerRequestBuilder addContainerRequest(Resource capability, int count);
 
   void addToBlacklist(String resource);
@@ -129,4 +129,24 @@ public interface YarnAMClient extends Service {
    * @param id The ID returned by {@link YarnAMClient.ContainerRequestBuilder#apply()}.
    */
   void completeContainerRequest(String id);
+
+  /**
+   * Callback for allocate call.
+   */
+  interface AllocateHandler {
+
+    /**
+     * Invokes when a list of containers has been acquired from YARN
+     *
+     * @param launchers list of launchers for launching runnables
+     */
+    void acquired(List<? extends ProcessLauncher<YarnContainerInfo>> launchers);
+
+    /**
+     * Invokes when containers completed
+     *
+     * @param completed list of completed container status
+     */
+    void completed(List<YarnContainerStatus> completed);
+  }
 }


[03/24] twill git commit: (TWILL-90) Add new configuration options to set AM memory size

Posted by ch...@apache.org.
(TWILL-90) Add new configuration options to set AM memory size

This closes #36 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/4fb75281
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/4fb75281
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/4fb75281

Branch: refs/heads/site
Commit: 4fb7528110e5c242760c931923e485d530bf6c9b
Parents: 0cc3159
Author: Terence Yim <ch...@apache.org>
Authored: Thu Mar 16 14:27:25 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Thu Mar 16 16:13:24 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/twill/api/Configs.java | 22 ++++++++++++++++++++
 .../org/apache/twill/internal/Constants.java    |  7 -------
 .../internal/yarn/Hadoop20YarnAppClient.java    |  8 +++----
 .../internal/yarn/Hadoop21YarnAppClient.java    | 10 ++++-----
 .../apache/twill/yarn/YarnTwillPreparer.java    |  6 ++++--
 5 files changed, 35 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/4fb75281/twill-api/src/main/java/org/apache/twill/api/Configs.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/Configs.java b/twill-api/src/main/java/org/apache/twill/api/Configs.java
index ecc26d2..570eafe 100644
--- a/twill-api/src/main/java/org/apache/twill/api/Configs.java
+++ b/twill-api/src/main/java/org/apache/twill/api/Configs.java
@@ -67,6 +67,17 @@ public final class Configs {
      */
     public static final String LOCATION_CACHE_ANTIQUE_EXPIRY_MS = "twill.location.cache.antique.expiry.ms";
 
+    /**
+     * Size in MB for the memory size of the YARN application master container.
+     */
+    public static final String YARN_AM_MEMORY_MB = "twill.yarn.am.memory.mb";
+
+    /**
+     * Size in MB for the reserved non-heap memory size for the YARN application Java process.
+     * The actual reserved memory size is limited by the {@link #HEAP_RESERVED_MIN_RATIO}.
+     */
+    public static final String YARN_AM_RESERVED_MEMORY_MB = "twill.yarn.am.reserved.memory.mb";
+
     private Keys() {
     }
   }
@@ -96,6 +107,17 @@ public final class Configs {
      */
     public static final long LOCATION_CACHE_ANTIQUE_EXPIRY_MS = TimeUnit.MINUTES.toMillis(5);
 
+    /**
+     * Default AM container memory size.
+     */
+    public static final int YARN_AM_MEMORY_MB = 512;
+
+    /**
+     * Default AM JVM reserved memory.
+     */
+    public static final int YARN_AM_RESERVED_MEMORY_MB = 150;
+
+
     private Defaults() {
     }
   }

http://git-wip-us.apache.org/repos/asf/twill/blob/4fb75281/twill-common/src/main/java/org/apache/twill/internal/Constants.java
----------------------------------------------------------------------
diff --git a/twill-common/src/main/java/org/apache/twill/internal/Constants.java b/twill-common/src/main/java/org/apache/twill/internal/Constants.java
index 569b396..6e799d5 100644
--- a/twill-common/src/main/java/org/apache/twill/internal/Constants.java
+++ b/twill-common/src/main/java/org/apache/twill/internal/Constants.java
@@ -37,13 +37,6 @@ public final class Constants {
    */
   public static final int CONSTRAINED_PROVISION_REQUEST_TIMEOUT = 5000;
 
-  public static final double HEAP_MIN_RATIO = 0.7d;
-
-  /** Memory size of AM. */
-  public static final int APP_MASTER_MEMORY_MB = 512;
-
-  public static final int APP_MASTER_RESERVED_MEMORY_MB = 150;
-
   /** Command names for the restart runnable instances. */
   public static final String RESTART_ALL_RUNNABLE_INSTANCES = "restartAllRunnableInstances";
   public static final String RESTART_RUNNABLES_INSTANCES = "restartRunnablesInstances";

http://git-wip-us.apache.org/repos/asf/twill/blob/4fb75281/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
index bb494d4..0ea58c0 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
@@ -18,7 +18,6 @@
 package org.apache.twill.internal.yarn;
 
 import com.google.common.base.Throwables;
-import com.google.common.util.concurrent.AbstractIdleService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -37,8 +36,8 @@ import org.apache.hadoop.yarn.client.YarnClient;
 import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.twill.api.Configs;
 import org.apache.twill.api.TwillSpecification;
-import org.apache.twill.internal.Constants;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.ProcessLauncher;
 import org.apache.twill.internal.appmaster.ApplicationMasterInfo;
@@ -96,10 +95,11 @@ public final class Hadoop20YarnAppClient implements YarnAppClient {
       appSubmissionContext.setQueue(schedulerQueue);
     }
 
-    // TODO: Make it adjustable through TwillSpec (TWILL-90)
+
+    int memoryMB = configuration.getInt(Configs.Keys.YARN_AM_MEMORY_MB, Configs.Defaults.YARN_AM_MEMORY_MB);
     // Set the resource requirement for AM
     Resource amResource = Records.newRecord(Resource.class);
-    amResource.setMemory(Constants.APP_MASTER_MEMORY_MB);
+    amResource.setMemory(memoryMB);
     final Resource capability = adjustMemory(response, amResource);
     ApplicationMasterInfo appMasterInfo = new ApplicationMasterInfo(appId, capability.getMemory(), 1);
 

http://git-wip-us.apache.org/repos/asf/twill/blob/4fb75281/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
index f24630f..0a64178 100644
--- a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
+++ b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
@@ -18,7 +18,6 @@
 package org.apache.twill.internal.yarn;
 
 import com.google.common.base.Throwables;
-import com.google.common.util.concurrent.AbstractIdleService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -34,8 +33,8 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.twill.api.Configs;
 import org.apache.twill.api.TwillSpecification;
-import org.apache.twill.internal.Constants;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.ProcessLauncher;
 import org.apache.twill.internal.appmaster.ApplicationMasterInfo;
@@ -55,6 +54,7 @@ import javax.annotation.Nullable;
  * Apache Hadoop 2.1 and beyond.
  * </p>
  */
+@SuppressWarnings("unused")
 public final class Hadoop21YarnAppClient implements YarnAppClient {
 
   private static final Logger LOG = LoggerFactory.getLogger(Hadoop21YarnAppClient.class);
@@ -90,9 +90,9 @@ public final class Hadoop21YarnAppClient implements YarnAppClient {
       appSubmissionContext.setQueue(schedulerQueue);
     }
 
-    // TODO: Make it adjustable through TwillSpec (TWILL-90)
     // Set the resource requirement for AM
-    final Resource capability = adjustMemory(response, Resource.newInstance(Constants.APP_MASTER_MEMORY_MB, 1));
+    int memoryMB = configuration.getInt(Configs.Keys.YARN_AM_MEMORY_MB, Configs.Defaults.YARN_AM_MEMORY_MB);
+    final Resource capability = adjustMemory(response, Resource.newInstance(memoryMB, 1));
     ApplicationMasterInfo appMasterInfo = new ApplicationMasterInfo(appId, capability.getMemory(),
                                                                     capability.getVirtualCores());
 
@@ -181,7 +181,7 @@ public final class Hadoop21YarnAppClient implements YarnAppClient {
     private final YarnClient yarnClient;
     private final ApplicationId appId;
 
-    public ProcessControllerImpl(YarnClient yarnClient, ApplicationId appId) {
+    ProcessControllerImpl(YarnClient yarnClient, ApplicationId appId) {
       this.yarnClient = yarnClient;
       this.appId = appId;
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/4fb75281/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index 6fbbf93..d9e70fd 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -404,9 +404,11 @@ final class YarnTwillPreparer implements TwillPreparer {
             //     org.apache.twill.internal.appmaster.ApplicationMasterMain
             //     false
 
+            int reservedMemoryMB = yarnConfig.getInt(Configs.Keys.YARN_AM_RESERVED_MEMORY_MB,
+                                                     Configs.Defaults.YARN_AM_RESERVED_MEMORY_MB);
             int memory = Resources.computeMaxHeapSize(appMasterInfo.getMemoryMB(),
-                                                      Constants.APP_MASTER_RESERVED_MEMORY_MB,
-                                                      Constants.HEAP_MIN_RATIO);
+                                                      reservedMemoryMB,
+                                                      minHeapRatio);
             return launcher.prepareLaunch(ImmutableMap.<String, String>of(), localFiles.values(), credentials)
               .addCommand(
                 "$JAVA_HOME/bin/java",


[19/24] twill git commit: (TWILL-230) Get resource report based on the caller user

Posted by ch...@apache.org.
(TWILL-230) Get resource report based on the caller user

- Also by default get the resource report from the tracking url, then fall back to the original tracking url.

This closes #53 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: c310b694582fb31eca4ba6f217254cb853a65a7e
Parents: 10fff16
Author: Terence Yim <ch...@apache.org>
Authored: Mon Apr 3 16:49:52 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Apr 3 20:55:39 2017 -0700

----------------------------------------------------------------------
 .../yarn/Hadoop20YarnApplicationReport.java     |  4 +-
 .../apache/twill/yarn/ResourceReportClient.java | 28 ++++++----
 .../apache/twill/yarn/YarnTwillController.java  | 56 +++++++++++++++-----
 .../twill/yarn/ResourceReportTestRun.java       | 47 +++++++++++++---
 twill-yarn/src/test/resources/logback-test.xml  |  1 +
 5 files changed, 104 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/c310b694/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnApplicationReport.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnApplicationReport.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnApplicationReport.java
index 6c1b764..8d6e2df 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnApplicationReport.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnApplicationReport.java
@@ -77,12 +77,12 @@ public final class Hadoop20YarnApplicationReport implements YarnApplicationRepor
 
   @Override
   public String getTrackingUrl() {
-    return report.getTrackingUrl();
+    return "http://" + report.getTrackingUrl();
   }
 
   @Override
   public String getOriginalTrackingUrl() {
-    return report.getOriginalTrackingUrl();
+    return "http://" + report.getOriginalTrackingUrl();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/twill/blob/c310b694/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java b/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
index 3d5bcf3..fb8b7e8 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
@@ -25,9 +25,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
+import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.Reader;
 import java.net.URL;
+import java.util.List;
 
 /**
  * Package private class to get {@link ResourceReport} from the application master.
@@ -36,10 +38,10 @@ final class ResourceReportClient {
   private static final Logger LOG = LoggerFactory.getLogger(ResourceReportClient.class);
 
   private final ResourceReportAdapter reportAdapter;
-  private final URL resourceUrl;
+  private final List<URL> resourceUrls;
 
-  ResourceReportClient(URL resourceUrl) {
-    this.resourceUrl = resourceUrl;
+  ResourceReportClient(List<URL> resourceUrls) {
+    this.resourceUrls = resourceUrls;
     this.reportAdapter = ResourceReportAdapter.create();
   }
 
@@ -48,16 +50,20 @@ final class ResourceReportClient {
    * @return A {@link ResourceReport} or {@code null} if failed to fetch the report.
    */
   public ResourceReport get() {
-    try {
-      Reader reader = new BufferedReader(new InputStreamReader(resourceUrl.openStream(), Charsets.UTF_8));
+    for (URL url : resourceUrls) {
       try {
-        return reportAdapter.fromJson(reader);
-      } finally {
-        Closeables.closeQuietly(reader);
+        Reader reader = new BufferedReader(new InputStreamReader(url.openStream(), Charsets.UTF_8));
+        try {
+          LOG.trace("Report returned by {}", url);
+          return reportAdapter.fromJson(reader);
+        } finally {
+          Closeables.closeQuietly(reader);
+        }
+      } catch (IOException e) {
+        // Just log a trace as it's ok to not able to fetch resource report
+        LOG.trace("Exception raised when getting resource report from {}.", url, e);
       }
-    } catch (Exception e) {
-      LOG.error("Exception getting resource report from {}.", resourceUrl, e);
-      return null;
     }
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/c310b694/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
index 1945731..6ea7d8f 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
@@ -43,10 +43,12 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.net.URI;
+import java.net.MalformedURLException;
 import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -66,7 +68,6 @@ final class YarnTwillController extends AbstractTwillController implements Twill
   private final TimeUnit startTimeoutUnit;
   private volatile ApplicationMasterLiveNodeData amLiveNodeData;
   private ProcessController<YarnApplicationReport> processController;
-  private ResourceReportClient resourcesClient;
 
   // Thread for polling yarn for application status if application got ZK session expire.
   // Only used by the instanceUpdate/Delete method, which is from serialized call from ZK callback.
@@ -101,7 +102,6 @@ final class YarnTwillController extends AbstractTwillController implements Twill
     this.startTimeoutUnit = startTimeoutUnit;
   }
 
-
   /**
    * Sends a message to application to notify the secure store has be updated.
    */
@@ -140,14 +140,6 @@ final class YarnTwillController extends AbstractTwillController implements Twill
       if (state != YarnApplicationState.RUNNING) {
         LOG.info("Yarn application {} {} is not in running state. Shutting down controller.", appName, appId);
         forceShutDown();
-      } else {
-        try {
-          URL resourceUrl = URI.create(String.format("http://%s:%d", report.getHost(), report.getRpcPort()))
-                               .resolve(TrackerService.PATH).toURL();
-          resourcesClient = new ResourceReportClient(resourceUrl);
-        } catch (IOException e) {
-          resourcesClient = null;
-        }
       }
     } catch (Exception e) {
       throw Throwables.propagate(e);
@@ -322,7 +314,45 @@ final class YarnTwillController extends AbstractTwillController implements Twill
 
   @Override
   public ResourceReport getResourceReport() {
-    // in case the user calls this before starting, return null
+    // Only has resource report if the app is running.
+    if (state() != State.RUNNING) {
+      return null;
+    }
+    ResourceReportClient resourcesClient = getResourcesClient();
     return (resourcesClient == null) ? null : resourcesClient.get();
   }
+
+  /**
+   * Returns the {@link ResourceReportClient} for fetching resource report from the AM.
+   * It first consults the RM for the tracking URL and get the resource report from there.
+   */
+  @Nullable
+  private ResourceReportClient getResourcesClient() {
+    YarnApplicationReport report = processController.getReport();
+    List<URL> urls = new ArrayList<>(2);
+
+    // Try getting the report from the proxy tracking URL as well as the original tracking URL directly
+    // This is mainly to workaround for unit-test that the proxy tracking URL doesn't work well with local address.
+    for (String url : Arrays.asList(report.getTrackingUrl(), report.getOriginalTrackingUrl())) {
+      if (url != null && !url.equals("N/A")) {
+        try {
+          URL trackingUrl = new URL(url);
+          String path = trackingUrl.getPath();
+          if (path.endsWith("/")) {
+            path = path.substring(0, path.length() - 1);
+          }
+          urls.add(new URL(trackingUrl.getProtocol(), trackingUrl.getHost(),
+                           trackingUrl.getPort(), path + TrackerService.PATH));
+        } catch (MalformedURLException e) {
+          LOG.debug("Invalid tracking URL {} from YARN application report for {}:{}", url, appName, getRunId());
+        }
+      }
+    }
+
+    if (urls.isEmpty()) {
+      return null;
+    }
+
+    return new ResourceReportClient(urls);
+  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/c310b694/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
index 32e1fd6..a61880f 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
@@ -18,15 +18,18 @@
 package org.apache.twill.yarn;
 
 import com.google.common.base.Charsets;
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
 import com.google.common.io.LineReader;
+import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.twill.api.ResourceReport;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.TwillApplication;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunResources;
 import org.apache.twill.api.TwillRunner;
+import org.apache.twill.api.TwillRunnerService;
 import org.apache.twill.api.TwillSpecification;
 import org.apache.twill.api.logging.PrinterLogHandler;
 import org.apache.twill.common.Threads;
@@ -159,7 +162,7 @@ public final class ResourceReportTestRun extends BaseYarnTest {
     Iterable<Discoverable> echoServices = controller.discoverService("echo");
     Assert.assertTrue(waitForSize(echoServices, 2, 120));
     // check that we have 2 runnables.
-    ResourceReport report = controller.getResourceReport();
+    ResourceReport report = getResourceReport(controller, 10000);
     Assert.assertEquals(2, report.getRunnableResources("BuggyServer").size());
 
     // cause a divide by 0 in one server
@@ -175,7 +178,7 @@ public final class ResourceReportTestRun extends BaseYarnTest {
     // takes some time for app master to find out the container completed...
     int count = 0;
     while (count < 100) {
-      report = controller.getResourceReport();
+      report = getResourceReport(controller, 10000);
       // check that we have 1 runnable, not 2.
       if (report.getRunnableResources("BuggyServer").size() == 1) {
         break;
@@ -216,7 +219,7 @@ public final class ResourceReportTestRun extends BaseYarnTest {
     // wait for 3 echo servers to come up
     Iterable<Discoverable> echoServices = controller.discoverService("echo");
     Assert.assertTrue(waitForSize(echoServices, 3, 120));
-    ResourceReport report = controller.getResourceReport();
+    ResourceReport report = getResourceReport(controller, 10000);
     // make sure resources for echo1 and echo2 are there
     Map<String, Collection<TwillRunResources>> usedResources = report.getResources();
     Assert.assertEquals(2, usedResources.keySet().size());
@@ -226,10 +229,10 @@ public final class ResourceReportTestRun extends BaseYarnTest {
     waitForSize(new Iterable<String>() {
       @Override
       public Iterator<String> iterator() {
-        return controller.getResourceReport().getServices().iterator();
+        return getResourceReport(controller, 10000).getServices().iterator();
       }
     }, 3, 120);
-    report = controller.getResourceReport();
+    report = getResourceReport(controller, 10000);
     Assert.assertEquals(ImmutableSet.of("echo", "echo1", "echo2"), ImmutableSet.copyOf(report.getServices()));
 
     Collection<TwillRunResources> echo1Resources = usedResources.get("echo1");
@@ -252,7 +255,7 @@ public final class ResourceReportTestRun extends BaseYarnTest {
     controller.changeInstances("echo1", 1).get(60, TimeUnit.SECONDS);
     echoServices = controller.discoverService("echo1");
     Assert.assertTrue(waitForSize(echoServices, 1, 60));
-    report = controller.getResourceReport();
+    report = getResourceReport(controller, 10000);
 
     // make sure resources for echo1 and echo2 are there
     usedResources = report.getResources();
@@ -276,8 +279,40 @@ public final class ResourceReportTestRun extends BaseYarnTest {
       Assert.assertEquals(512, resources.getMemoryMB());
     }
 
+    // Create a new TwillRunner, it should be able to get the same resource report
+    TwillRunnerService newRunnerService = TWILL_TESTER.createTwillRunnerService();
+    newRunnerService.start();
+    try {
+      TwillController newController = newRunnerService.lookup("ResourceApplication", controller.getRunId());
+      // Get the controller of the application
+      int trials = 60;
+      while (newController == null && trials-- > 0) {
+        TimeUnit.SECONDS.sleep(1);
+        newController = newRunnerService.lookup("ResourceApplication", controller.getRunId());
+      }
+      Assert.assertNotNull(newController);
+
+      ResourceReport newReport = getResourceReport(newController, 10000);
+      Assert.assertEquals(report.getResources(), newReport.getResources());
+
+    } finally {
+      newRunnerService.stop();
+    }
+
     controller.terminate().get(120, TimeUnit.SECONDS);
     // Sleep a bit before exiting.
     TimeUnit.SECONDS.sleep(2);
   }
+
+  private ResourceReport getResourceReport(TwillController controller, long timeoutMillis) {
+    ResourceReport report = controller.getResourceReport();
+    Stopwatch stopwatch = new Stopwatch();
+    while (report == null && stopwatch.elapsedMillis() < timeoutMillis) {
+      Uninterruptibles.sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
+      report = controller.getResourceReport();
+    }
+
+    Assert.assertNotNull(report);
+    return report;
+  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/c310b694/twill-yarn/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/resources/logback-test.xml b/twill-yarn/src/test/resources/logback-test.xml
index 4bcdb42..2b210cb 100644
--- a/twill-yarn/src/test/resources/logback-test.xml
+++ b/twill-yarn/src/test/resources/logback-test.xml
@@ -24,6 +24,7 @@ limitations under the License.
         </encoder>
     </appender>
 
+    <logger name="org.mortbay" level="OFF" />
     <logger name="org.apache.twill" level="DEBUG" />
 
     <root level="WARN">


[10/24] twill git commit: (TWILL-194) Acquires KMS delegation token correctly

Posted by ch...@apache.org.
(TWILL-194) Acquires KMS delegation token correctly

- This is to workaround HDFS-10296 that if FileContext object is used
  to acquire delegation token, the KMS delegation token is not included

This closes #43 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/390dfabd
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/390dfabd
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/390dfabd

Branch: refs/heads/site
Commit: 390dfabd6d5c339b049736efdb031846380ae892
Parents: d2a503a
Author: Terence Yim <ch...@apache.org>
Authored: Mon Mar 27 10:58:54 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Mar 27 12:40:49 2017 -0700

----------------------------------------------------------------------
 .../apache/twill/internal/yarn/YarnUtils.java   | 53 +++++++++++---------
 1 file changed, 29 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/390dfabd/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
index ff8f4bb..e931144 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
@@ -22,9 +22,7 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.io.DataInputByteBuffer;
@@ -53,9 +51,11 @@ import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
 
 /**
  * Collection of helper methods to simplify YARN calls.
@@ -160,26 +160,19 @@ public class YarnUtils {
       return ImmutableList.of();
     }
 
-    LocationFactory factory = unwrap(locationFactory);
-    String renewer = getYarnTokenRenewer(config);
-    List<Token<?>> tokens = ImmutableList.of();
+    FileSystem fileSystem = getFileSystem(locationFactory, config);
 
-    if (factory instanceof HDFSLocationFactory) {
-      FileSystem fs = ((HDFSLocationFactory) factory).getFileSystem();
-      Token<?>[] fsTokens = fs.addDelegationTokens(renewer, credentials);
-      if (fsTokens != null) {
-        tokens = ImmutableList.copyOf(fsTokens);
-      }
-    } else if (factory instanceof FileContextLocationFactory) {
-      FileContext fc = ((FileContextLocationFactory) factory).getFileContext();
-      tokens = fc.getDelegationTokens(new Path(locationFactory.create("/").toURI()), renewer);
+    if (fileSystem == null) {
+      LOG.warn("Unexpected: LocationFactory is neither FileContextLocationFactory nor HDFSLocationFactory.");
+      return ImmutableList.of();
     }
 
-    for (Token<?> token : tokens) {
-      credentials.addToken(token.getService(), token);
-    }
+    String renewer = YarnUtils.getYarnTokenRenewer(config);
+
+    Token<?>[] tokens = fileSystem.addDelegationTokens(renewer, credentials);
+    LOG.debug("Added HDFS DelegationTokens: {}", Arrays.toString(tokens));
 
-    return ImmutableList.copyOf(tokens);
+    return tokens == null ? ImmutableList.<Token<?>>of() : ImmutableList.copyOf(tokens);
   }
 
   /**
@@ -298,6 +291,7 @@ public class YarnUtils {
     }
 
     try {
+      //noinspection unchecked
       return (T) Class.forName(className).newInstance();
     } catch (Exception e) {
       throw Throwables.propagate(e);
@@ -319,14 +313,25 @@ public class YarnUtils {
   }
 
   /**
-   * Unwraps the given {@link LocationFactory} and returns the inner most {@link LocationFactory} which is not
-   * a {@link ForwardingLocationFactory}.
+   * Gets the Hadoop {@link FileSystem} from LocationFactory.
+   *
+   * @return the Hadoop {@link FileSystem} that represents the filesystem used by the given {@link LocationFactory};
+   *         {@code null} will be returned if unable to determine the {@link FileSystem}.
    */
-  private static LocationFactory unwrap(LocationFactory locationFactory) {
-    while (locationFactory instanceof ForwardingLocationFactory) {
-      locationFactory = ((ForwardingLocationFactory) locationFactory).getDelegate();
+  @Nullable
+  private static FileSystem getFileSystem(LocationFactory locationFactory, Configuration config) throws IOException {
+    if (locationFactory instanceof HDFSLocationFactory) {
+      return ((HDFSLocationFactory) locationFactory).getFileSystem();
+    }
+    if (locationFactory instanceof ForwardingLocationFactory) {
+      return getFileSystem(((ForwardingLocationFactory) locationFactory).getDelegate(), config);
+    }
+    // Due to HDFS-10296, for encrypted file systems, FileContext does not acquire the KMS delegation token
+    // Since we know we are in Yarn, it is safe to get the FileSystem directly, bypassing LocationFactory.
+    if (locationFactory instanceof FileContextLocationFactory) {
+      return FileSystem.get(config);
     }
-    return locationFactory;
+    return null;
   }
 
   private YarnUtils() {


[12/24] twill git commit: (TWILL-227) Disabling caching of FileSystem instance when getting delegation token

Posted by ch...@apache.org.
(TWILL-227) Disabling caching of FileSystem instance when getting delegation token

- Allows getting delegation token for different users without leaking
memory.
  - The FileSystem.get() by default will cache all FileSystem instances
until end of process.

This closes #46 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/4e1cae3d
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/4e1cae3d
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/4e1cae3d

Branch: refs/heads/site
Commit: 4e1cae3de294489e42959e0c663cf6ea69fa6ccb
Parents: 7f34871
Author: Terence Yim <ch...@apache.org>
Authored: Mon Mar 27 16:37:07 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Tue Mar 28 16:28:13 2017 -0700

----------------------------------------------------------------------
 .../apache/twill/internal/yarn/YarnUtils.java   | 30 ++++++++++++--------
 1 file changed, 18 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/4e1cae3d/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
index 3c4c270..c0aeb0c 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
@@ -159,19 +159,18 @@ public class YarnUtils {
       return ImmutableList.of();
     }
 
-    FileSystem fileSystem = getFileSystem(locationFactory, config);
-
-    if (fileSystem == null) {
-      LOG.warn("Unexpected: LocationFactory is not backed by FileContextLocationFactory");
-      return ImmutableList.of();
-    }
+    try (FileSystem fileSystem = getFileSystem(locationFactory)) {
+      if (fileSystem == null) {
+        return ImmutableList.of();
+      }
 
-    String renewer = YarnUtils.getYarnTokenRenewer(config);
+      String renewer = YarnUtils.getYarnTokenRenewer(config);
 
-    Token<?>[] tokens = fileSystem.addDelegationTokens(renewer, credentials);
-    LOG.debug("Added HDFS DelegationTokens: {}", Arrays.toString(tokens));
+      Token<?>[] tokens = fileSystem.addDelegationTokens(renewer, credentials);
+      LOG.debug("Added HDFS DelegationTokens: {}", Arrays.toString(tokens));
 
-    return tokens == null ? ImmutableList.<Token<?>>of() : ImmutableList.copyOf(tokens);
+      return tokens == null ? ImmutableList.<Token<?>>of() : ImmutableList.copyOf(tokens);
+    }
   }
 
   /**
@@ -318,15 +317,22 @@ public class YarnUtils {
    *         {@code null} will be returned if unable to determine the {@link FileSystem}.
    */
   @Nullable
-  private static FileSystem getFileSystem(LocationFactory locationFactory, Configuration config) throws IOException {
+  private static FileSystem getFileSystem(LocationFactory locationFactory) throws IOException {
     if (locationFactory instanceof ForwardingLocationFactory) {
-      return getFileSystem(((ForwardingLocationFactory) locationFactory).getDelegate(), config);
+      return getFileSystem(((ForwardingLocationFactory) locationFactory).getDelegate());
     }
     // Due to HDFS-10296, for encrypted file systems, FileContext does not acquire the KMS delegation token
     // Since we know we are in Yarn, it is safe to get the FileSystem directly, bypassing LocationFactory.
     if (locationFactory instanceof FileContextLocationFactory) {
+      // Disable caching of FileSystem object, as the FileSystem object is only used to get delegation token for the
+      // current user. Caching it may causes leaking of FileSystem object if the method is called with different users.
+      Configuration config = new Configuration(((FileContextLocationFactory) locationFactory).getConfiguration());
+      String scheme = FileSystem.getDefaultUri(config).getScheme();
+      config.set(String.format("fs.%s.impl.disable.cache", scheme), "true");
       return FileSystem.get(config);
     }
+
+    LOG.warn("Unexpected: LocationFactory is not backed by FileContextLocationFactory");
     return null;
   }
 


[04/24] twill git commit: (TWILL-119) Make YarnAppClient supports HA RM

Posted by ch...@apache.org.
(TWILL-119) Make YarnAppClient supports HA RM

- HA RM is added since Hadoop 2.3.

This closes #38 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: a3e4d38e42a464bc35d7b1dec68d7e7249c8e00b
Parents: 4fb7528
Author: Terence Yim <ch...@apache.org>
Authored: Fri Mar 17 14:17:47 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Fri Mar 17 17:41:37 2017 -0700

----------------------------------------------------------------------
 pom.xml                                         |   4 +
 .../internal/yarn/Hadoop20YarnAppClient.java    | 116 +++++++++--------
 .../internal/yarn/Hadoop21YarnAppClient.java    | 129 ++++++++++---------
 .../internal/yarn/Hadoop23YarnAppClient.java    | 111 ++++++++++++++++
 .../yarn/VersionDetectYarnAMClientFactory.java  |   2 +-
 .../yarn/VersionDetectYarnAppClientFactory.java |  25 ++--
 .../apache/twill/internal/yarn/YarnUtils.java   |  10 +-
 7 files changed, 268 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/a3e4d38e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 55f055b..b641b49 100644
--- a/pom.xml
+++ b/pom.xml
@@ -558,6 +558,7 @@
                                     <sources>
                                         <source>src/main/hadoop21</source>
                                         <source>src/main/hadoop22</source>
+                                        <source>src/main/hadoop23</source>
                                     </sources>
                                 </configuration>
                             </execution>
@@ -600,6 +601,7 @@
                                     <sources>
                                         <source>src/main/hadoop21</source>
                                         <source>src/main/hadoop22</source>
+                                        <source>src/main/hadoop23</source>
                                     </sources>
                                 </configuration>
                             </execution>
@@ -656,6 +658,7 @@
                                     <sources>
                                         <source>src/main/hadoop21</source>
                                         <source>src/main/hadoop22</source>
+                                        <source>src/main/hadoop23</source>
                                     </sources>
                                 </configuration>
                             </execution>
@@ -698,6 +701,7 @@
                                     <sources>
                                         <source>src/main/hadoop21</source>
                                         <source>src/main/hadoop22</source>
+                                        <source>src/main/hadoop23</source>
                                     </sources>
                                 </configuration>
                             </execution>

http://git-wip-us.apache.org/repos/asf/twill/blob/a3e4d38e/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
index 0ea58c0..787f8a4 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
@@ -17,7 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import com.google.common.base.Throwables;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -46,6 +45,7 @@ import org.apache.twill.internal.appmaster.ApplicationSubmitter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.List;
 import javax.annotation.Nullable;
@@ -58,6 +58,7 @@ import javax.annotation.Nullable;
  * Apache Hadoop 2.0.
  * </p>
  */
+@SuppressWarnings("unused")
 public final class Hadoop20YarnAppClient implements YarnAppClient {
 
   private static final Logger LOG = LoggerFactory.getLogger(Hadoop20YarnAppClient.class);
@@ -80,50 +81,56 @@ public final class Hadoop20YarnAppClient implements YarnAppClient {
   @Override
   public ProcessLauncher<ApplicationMasterInfo> createLauncher(TwillSpecification twillSpec,
                                                                @Nullable String schedulerQueue) throws Exception {
-    final YarnClient yarnClient = createYarnClient();
-    // Request for new application
-    final GetNewApplicationResponse response = yarnClient.getNewApplication();
-    final ApplicationId appId = response.getApplicationId();
-
-    // Setup the context for application submission
-    final ApplicationSubmissionContext appSubmissionContext = Records.newRecord(ApplicationSubmissionContext.class);
-    appSubmissionContext.setApplicationId(appId);
-    appSubmissionContext.setApplicationName(twillSpec.getName());
-    appSubmissionContext.setUser(user);
-
-    if (schedulerQueue != null) {
-      appSubmissionContext.setQueue(schedulerQueue);
-    }
+    YarnClient yarnClient = createYarnClient();
+    try {
+      // Request for new application
+      final GetNewApplicationResponse response = yarnClient.getNewApplication();
+      final ApplicationId appId = response.getApplicationId();
+
+      // Setup the context for application submission
+      final ApplicationSubmissionContext appSubmissionContext = Records.newRecord(ApplicationSubmissionContext.class);
+      appSubmissionContext.setApplicationId(appId);
+      appSubmissionContext.setApplicationName(twillSpec.getName());
+      appSubmissionContext.setUser(user);
+
+      if (schedulerQueue != null) {
+        appSubmissionContext.setQueue(schedulerQueue);
+      }
 
 
-    int memoryMB = configuration.getInt(Configs.Keys.YARN_AM_MEMORY_MB, Configs.Defaults.YARN_AM_MEMORY_MB);
-    // Set the resource requirement for AM
-    Resource amResource = Records.newRecord(Resource.class);
-    amResource.setMemory(memoryMB);
-    final Resource capability = adjustMemory(response, amResource);
-    ApplicationMasterInfo appMasterInfo = new ApplicationMasterInfo(appId, capability.getMemory(), 1);
-
-    ApplicationSubmitter submitter = new ApplicationSubmitter() {
-
-      @Override
-      public ProcessController<YarnApplicationReport> submit(YarnLaunchContext launchContext) {
-        ContainerLaunchContext context = launchContext.getLaunchContext();
-        addRMToken(context, yarnClient);
-        context.setUser(appSubmissionContext.getUser());
-        context.setResource(adjustMemory(response, capability));
-        appSubmissionContext.setAMContainerSpec(context);
-
-        try {
-          yarnClient.submitApplication(appSubmissionContext);
-          return new ProcessControllerImpl(yarnClient, appId);
-        } catch (YarnRemoteException e) {
-          LOG.error("Failed to submit application {}", appId, e);
-          throw Throwables.propagate(e);
+      int memoryMB = configuration.getInt(Configs.Keys.YARN_AM_MEMORY_MB, Configs.Defaults.YARN_AM_MEMORY_MB);
+      // Set the resource requirement for AM
+      Resource amResource = Records.newRecord(Resource.class);
+      amResource.setMemory(memoryMB);
+      final Resource capability = adjustMemory(response, amResource);
+      ApplicationMasterInfo appMasterInfo = new ApplicationMasterInfo(appId, capability.getMemory(), 1);
+
+      ApplicationSubmitter submitter = new ApplicationSubmitter() {
+
+        @Override
+        public ProcessController<YarnApplicationReport> submit(YarnLaunchContext launchContext) {
+          YarnClient yarnClient = createYarnClient();
+          try {
+            ContainerLaunchContext context = launchContext.getLaunchContext();
+            addRMToken(context, yarnClient);
+            context.setUser(appSubmissionContext.getUser());
+            context.setResource(adjustMemory(response, capability));
+            appSubmissionContext.setAMContainerSpec(context);
+
+            yarnClient.submitApplication(appSubmissionContext);
+            return new ProcessControllerImpl(yarnClient, appId);
+          } catch (YarnRemoteException e) {
+            throw new RuntimeException("Failed to submit application " + appId, e);
+          } finally {
+            yarnClient.stop();
+          }
         }
-      }
-    };
+      };
 
-    return new ApplicationMasterProcessLauncher(appMasterInfo, submitter);
+      return new ApplicationMasterProcessLauncher(appMasterInfo, submitter);
+    } finally {
+      yarnClient.stop();
+    }
   }
 
   private Resource adjustMemory(GetNewApplicationResponse response, Resource capability) {
@@ -152,22 +159,21 @@ public final class Hadoop20YarnAppClient implements YarnAppClient {
         yarnClient.getRMDelegationToken(new Text(YarnUtils.getYarnTokenRenewer(config))),
         YarnUtils.getRMAddress(config));
 
-      LOG.info("Added RM delegation token {}", token);
+      LOG.debug("Added RM delegation token {}", token);
       credentials.addToken(token.getService(), token);
 
       context.setContainerTokens(YarnUtils.encodeCredentials(credentials));
-
-    } catch (Exception e) {
-      LOG.error("Fails to create credentials.", e);
-      throw Throwables.propagate(e);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to acquire RM delegation token", e);
     }
   }
 
-  private <T extends TokenIdentifier> Token<T> convertToken(DelegationToken protoToken, InetSocketAddress serviceAddr) {
-    Token<T> token = new Token<T>(protoToken.getIdentifier().array(),
-                                  protoToken.getPassword().array(),
-                                  new Text(protoToken.getKind()),
-                                  new Text(protoToken.getService()));
+  private <T extends TokenIdentifier> Token<T> convertToken(DelegationToken protoToken,
+                                                            @Nullable InetSocketAddress serviceAddr) {
+    Token<T> token = new Token<>(protoToken.getIdentifier().array(),
+                                 protoToken.getPassword().array(),
+                                 new Text(protoToken.getKind()),
+                                 new Text(protoToken.getService()));
     if (serviceAddr != null) {
       SecurityUtil.setTokenService(token, serviceAddr);
     }
@@ -201,7 +207,7 @@ public final class Hadoop20YarnAppClient implements YarnAppClient {
     private final YarnClient yarnClient;
     private final ApplicationId appId;
 
-    public ProcessControllerImpl(YarnClient yarnClient, ApplicationId appId) {
+    ProcessControllerImpl(YarnClient yarnClient, ApplicationId appId) {
       this.yarnClient = yarnClient;
       this.appId = appId;
     }
@@ -211,8 +217,7 @@ public final class Hadoop20YarnAppClient implements YarnAppClient {
       try {
         return new Hadoop20YarnApplicationReport(yarnClient.getApplicationReport(appId));
       } catch (YarnRemoteException e) {
-        LOG.error("Failed to get application report {}", appId, e);
-        throw Throwables.propagate(e);
+        throw new RuntimeException("Failed to get application report for " + appId, e);
       }
     }
 
@@ -221,8 +226,7 @@ public final class Hadoop20YarnAppClient implements YarnAppClient {
       try {
         yarnClient.killApplication(appId);
       } catch (YarnRemoteException e) {
-        LOG.error("Failed to kill application {}", appId, e);
-        throw Throwables.propagate(e);
+        throw new RuntimeException("Failed to kill application " + appId, e);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/twill/blob/a3e4d38e/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
index 0a64178..aa14a75 100644
--- a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
+++ b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
@@ -17,7 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import com.google.common.base.Throwables;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -32,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.twill.api.Configs;
 import org.apache.twill.api.TwillSpecification;
@@ -43,6 +43,7 @@ import org.apache.twill.internal.appmaster.ApplicationSubmitter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.List;
 import javax.annotation.Nullable;
 
@@ -55,7 +56,7 @@ import javax.annotation.Nullable;
  * </p>
  */
 @SuppressWarnings("unused")
-public final class Hadoop21YarnAppClient implements YarnAppClient {
+public class Hadoop21YarnAppClient implements YarnAppClient {
 
   private static final Logger LOG = LoggerFactory.getLogger(Hadoop21YarnAppClient.class);
   private final Configuration configuration;
@@ -75,48 +76,54 @@ public final class Hadoop21YarnAppClient implements YarnAppClient {
   @Override
   public ProcessLauncher<ApplicationMasterInfo> createLauncher(TwillSpecification twillSpec,
                                                                @Nullable String schedulerQueue) throws Exception {
-    final YarnClient yarnClient = createYarnClient();
-    // Request for new application
-    YarnClientApplication application = yarnClient.createApplication();
-    final GetNewApplicationResponse response = application.getNewApplicationResponse();
-    final ApplicationId appId = response.getApplicationId();
-
-    // Setup the context for application submission
-    final ApplicationSubmissionContext appSubmissionContext = application.getApplicationSubmissionContext();
-    appSubmissionContext.setApplicationId(appId);
-    appSubmissionContext.setApplicationName(twillSpec.getName());
-
-    if (schedulerQueue != null) {
-      appSubmissionContext.setQueue(schedulerQueue);
-    }
+    YarnClient yarnClient = createYarnClient();
+    try {
+      // Request for new application
+      YarnClientApplication application = yarnClient.createApplication();
+      final GetNewApplicationResponse response = application.getNewApplicationResponse();
+      final ApplicationId appId = response.getApplicationId();
+
+      // Setup the context for application submission
+      final ApplicationSubmissionContext appSubmissionContext = application.getApplicationSubmissionContext();
+      appSubmissionContext.setApplicationId(appId);
+      appSubmissionContext.setApplicationName(twillSpec.getName());
+
+      if (schedulerQueue != null) {
+        appSubmissionContext.setQueue(schedulerQueue);
+      }
 
-    // Set the resource requirement for AM
-    int memoryMB = configuration.getInt(Configs.Keys.YARN_AM_MEMORY_MB, Configs.Defaults.YARN_AM_MEMORY_MB);
-    final Resource capability = adjustMemory(response, Resource.newInstance(memoryMB, 1));
-    ApplicationMasterInfo appMasterInfo = new ApplicationMasterInfo(appId, capability.getMemory(),
-                                                                    capability.getVirtualCores());
-
-    ApplicationSubmitter submitter = new ApplicationSubmitter() {
-      @Override
-      public ProcessController<YarnApplicationReport> submit(YarnLaunchContext context) {
-        ContainerLaunchContext launchContext = context.getLaunchContext();
-
-        addRMToken(launchContext, yarnClient);
-        appSubmissionContext.setAMContainerSpec(launchContext);
-        appSubmissionContext.setResource(capability);
-        appSubmissionContext.setMaxAppAttempts(2);
-
-        try {
-          yarnClient.submitApplication(appSubmissionContext);
-          return new ProcessControllerImpl(yarnClient, appId);
-        } catch (Exception e) {
-          LOG.error("Failed to submit application {}", appId, e);
-          throw Throwables.propagate(e);
+      // Set the resource requirement for AM
+      int memoryMB = configuration.getInt(Configs.Keys.YARN_AM_MEMORY_MB, Configs.Defaults.YARN_AM_MEMORY_MB);
+      final Resource capability = adjustMemory(response, Resource.newInstance(memoryMB, 1));
+      ApplicationMasterInfo appMasterInfo = new ApplicationMasterInfo(appId, capability.getMemory(),
+                                                                      capability.getVirtualCores());
+
+      ApplicationSubmitter submitter = new ApplicationSubmitter() {
+        @Override
+        public ProcessController<YarnApplicationReport> submit(YarnLaunchContext context) {
+          ContainerLaunchContext launchContext = context.getLaunchContext();
+
+          YarnClient yarnClient = createYarnClient();
+          try {
+            addRMToken(launchContext, yarnClient, appId);
+            appSubmissionContext.setAMContainerSpec(launchContext);
+            appSubmissionContext.setResource(capability);
+            appSubmissionContext.setMaxAppAttempts(2);
+
+            yarnClient.submitApplication(appSubmissionContext);
+            return new ProcessControllerImpl(appId);
+          } catch (YarnException | IOException e) {
+            throw new RuntimeException("Failed to submit application " + appId, e);
+          } finally {
+            yarnClient.stop();
+          }
         }
-      }
-    };
+      };
 
-    return new ApplicationMasterProcessLauncher(appMasterInfo, submitter);
+      return new ApplicationMasterProcessLauncher(appMasterInfo, submitter);
+    } finally {
+      yarnClient.stop();
+    }
   }
 
   private Resource adjustMemory(GetNewApplicationResponse response, Resource capability) {
@@ -130,7 +137,11 @@ public final class Hadoop21YarnAppClient implements YarnAppClient {
     return capability;
   }
 
-  private void addRMToken(ContainerLaunchContext context, YarnClient yarnClient) {
+  /**
+   * Adds RM delegation token to the given {@link ContainerLaunchContext} so that the AM can authenticate itself
+   * with RM using the delegation token.
+   */
+  protected void addRMToken(ContainerLaunchContext context, YarnClient yarnClient, ApplicationId appId) {
     if (!UserGroupInformation.isSecurityEnabled()) {
       return;
     }
@@ -143,14 +154,12 @@ public final class Hadoop21YarnAppClient implements YarnAppClient {
         yarnClient.getRMDelegationToken(new Text(YarnUtils.getYarnTokenRenewer(config))),
         YarnUtils.getRMAddress(config));
 
-      LOG.info("Added RM delegation token {}", token);
+      LOG.debug("Added RM delegation token {} for application {}", token, appId);
       credentials.addToken(token.getService(), token);
 
       context.setTokens(YarnUtils.encodeCredentials(credentials));
-
-    } catch (Exception e) {
-      LOG.error("Fails to create credentials.", e);
-      throw Throwables.propagate(e);
+    } catch (YarnException | IOException e) {
+      throw new RuntimeException("Failed to acquire RM delegation token", e);
     }
   }
 
@@ -164,7 +173,7 @@ public final class Hadoop21YarnAppClient implements YarnAppClient {
 
   @Override
   public ProcessController<YarnApplicationReport> createProcessController(ApplicationId appId) {
-    return new ProcessControllerImpl(createYarnClient(), appId);
+    return new ProcessControllerImpl(appId);
   }
 
   @Override
@@ -177,38 +186,40 @@ public final class Hadoop21YarnAppClient implements YarnAppClient {
     }
   }
 
-  private static final class ProcessControllerImpl implements ProcessController<YarnApplicationReport> {
-    private final YarnClient yarnClient;
+  private final class ProcessControllerImpl implements ProcessController<YarnApplicationReport> {
     private final ApplicationId appId;
 
-    ProcessControllerImpl(YarnClient yarnClient, ApplicationId appId) {
-      this.yarnClient = yarnClient;
+    ProcessControllerImpl(ApplicationId appId) {
       this.appId = appId;
     }
 
     @Override
     public YarnApplicationReport getReport() {
+      YarnClient yarnClient = createYarnClient();
       try {
         return new Hadoop21YarnApplicationReport(yarnClient.getApplicationReport(appId));
-      } catch (Exception e) {
-        LOG.error("Failed to get application report {}", appId, e);
-        throw Throwables.propagate(e);
+      } catch (YarnException | IOException e) {
+        throw new RuntimeException("Failed to get application report for " + appId, e);
+      } finally {
+        yarnClient.stop();
       }
     }
 
     @Override
     public void cancel() {
+      YarnClient yarnClient = createYarnClient();
       try {
         yarnClient.killApplication(appId);
-      } catch (Exception e) {
-        LOG.error("Failed to kill application {}", appId, e);
-        throw Throwables.propagate(e);
+      } catch (YarnException | IOException e) {
+        throw new RuntimeException("Failed to kill application " + appId, e);
+      } finally {
+        yarnClient.stop();
       }
     }
 
     @Override
     public void close() throws Exception {
-      yarnClient.stop();
+      // no-op
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/a3e4d38e/twill-yarn/src/main/hadoop23/org/apache/twill/internal/yarn/Hadoop23YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop23/org/apache/twill/internal/yarn/Hadoop23YarnAppClient.java b/twill-yarn/src/main/hadoop23/org/apache/twill/internal/yarn/Hadoop23YarnAppClient.java
new file mode 100644
index 0000000..97d2a64
--- /dev/null
+++ b/twill-yarn/src/main/hadoop23/org/apache/twill/internal/yarn/Hadoop23YarnAppClient.java
@@ -0,0 +1,111 @@
+/*
+ * 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.twill.internal.yarn;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Throwables;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.HAUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * <p>
+ * The service implementation of {@link YarnAppClient} for Apache Hadoop 2.3 and beyond.
+ *
+ * The {@link VersionDetectYarnAppClientFactory} class will decide to return instance of this class for
+ * Apache Hadoop 2.3 and beyond.
+ * </p>
+ */
+@SuppressWarnings("unused")
+public final class Hadoop23YarnAppClient extends Hadoop21YarnAppClient {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Hadoop23YarnAppClient.class);
+  private final Configuration configuration;
+
+  public Hadoop23YarnAppClient(Configuration configuration) {
+    super(configuration);
+    this.configuration = configuration;
+  }
+
+  /**
+   * Overrides parent method to adds RM delegation token to the given context. If YARN is running with HA RM,
+   * delegation tokens for each RM service will be added.
+   */
+  protected void addRMToken(ContainerLaunchContext context, YarnClient yarnClient, ApplicationId appId) {
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      return;
+    }
+
+    try {
+      Text renewer = new Text(UserGroupInformation.getCurrentUser().getShortUserName());
+      org.apache.hadoop.yarn.api.records.Token rmDelegationToken = yarnClient.getRMDelegationToken(renewer);
+
+      // The following logic is copied from ClientRMProxy.getRMDelegationTokenService, which is not available in
+      // YARN older than 2.4
+      List<String> services = new ArrayList<>();
+      if (HAUtil.isHAEnabled(configuration)) {
+        // If HA is enabled, we need to enumerate all RM hosts
+        // and add the corresponding service name to the token service
+        // Copy the yarn conf since we need to modify it to get the RM addresses
+        YarnConfiguration yarnConf = new YarnConfiguration(configuration);
+        for (String rmId : HAUtil.getRMHAIds(configuration)) {
+          yarnConf.set(YarnConfiguration.RM_HA_ID, rmId);
+          InetSocketAddress address = yarnConf.getSocketAddr(YarnConfiguration.RM_ADDRESS,
+                                                             YarnConfiguration.DEFAULT_RM_ADDRESS,
+                                                             YarnConfiguration.DEFAULT_RM_PORT);
+          services.add(SecurityUtil.buildTokenService(address).toString());
+        }
+      } else {
+        services.add(SecurityUtil.buildTokenService(YarnUtils.getRMAddress(configuration)).toString());
+      }
+
+      Credentials credentials = YarnUtils.decodeCredentials(context.getTokens());
+
+      // casting needed for later Hadoop version
+      @SuppressWarnings("RedundantCast")
+      Token<TokenIdentifier> token = ConverterUtils.convertFromYarn(rmDelegationToken, (InetSocketAddress) null);
+
+      token.setService(new Text(Joiner.on(',').join(services)));
+      credentials.addToken(new Text(token.getService()), token);
+
+      LOG.debug("Added RM delegation token {} for application {}", token, appId);
+      credentials.addToken(token.getService(), token);
+
+      context.setTokens(YarnUtils.encodeCredentials(credentials));
+
+    } catch (Exception e) {
+      throw Throwables.propagate(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/twill/blob/a3e4d38e/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAMClientFactory.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAMClientFactory.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAMClientFactory.java
index d932e77..943efb8 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAMClientFactory.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAMClientFactory.java
@@ -49,7 +49,7 @@ public final class VersionDetectYarnAMClientFactory implements YarnAMClientFacto
           clz = (Class<YarnAMClient>) Class.forName(clzName);
           break;
         default:
-          // Uses hadoop-2.2 class
+          // Uses hadoop-2.2 or above class
           clzName = getClass().getPackage().getName() + ".Hadoop22YarnAMClient";
           clz = (Class<YarnAMClient>) Class.forName(clzName);
           break;

http://git-wip-us.apache.org/repos/asf/twill/blob/a3e4d38e/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAppClientFactory.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAppClientFactory.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAppClientFactory.java
index 9f6e8a1..c8e88c9 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAppClientFactory.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/VersionDetectYarnAppClientFactory.java
@@ -29,18 +29,21 @@ public final class VersionDetectYarnAppClientFactory implements YarnAppClientFac
   @SuppressWarnings("unchecked")
   public YarnAppClient create(Configuration configuration) {
     try {
-      Class<YarnAppClient> clz;
-
-      if (YarnUtils.getHadoopVersion().equals(YarnUtils.HadoopVersions.HADOOP_20)) {
-        // Uses hadoop-2.0 class.
-        String clzName = getClass().getPackage().getName() + ".Hadoop20YarnAppClient";
-        clz = (Class<YarnAppClient>) Class.forName(clzName);
-      } else {
-        // Uses hadoop-2.1 class
-        String clzName = getClass().getPackage().getName() + ".Hadoop21YarnAppClient";
-        clz = (Class<YarnAppClient>) Class.forName(clzName);
+      String clzName;
+      switch (YarnUtils.getHadoopVersion()) {
+        case HADOOP_20:
+          clzName = getClass().getPackage().getName() + ".Hadoop20YarnAppClient";
+          break;
+        case HADOOP_21:
+        case HADOOP_22:
+          // 2.1 and 2.2 uses the same YarnAppClient
+          clzName = getClass().getPackage().getName() + ".Hadoop21YarnAppClient";
+          break;
+        default:
+          // 2.3 and above uses the 2.3 YarnAppClient to support RM HA
+          clzName = getClass().getPackage().getName() + ".Hadoop23YarnAppClient";
       }
-
+      Class<YarnAppClient> clz = (Class<YarnAppClient>) Class.forName(clzName);
       return clz.getConstructor(Configuration.class).newInstance(configuration);
 
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/twill/blob/a3e4d38e/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
index cb81bee..9574554 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
@@ -63,7 +63,8 @@ public class YarnUtils {
   public enum HadoopVersions {
     HADOOP_20,
     HADOOP_21,
-    HADOOP_22
+    HADOOP_22,
+    HADOOP_23
   }
 
   private static final Logger LOG = LoggerFactory.getLogger(YarnUtils.class);
@@ -236,7 +237,12 @@ public class YarnUtils {
       Class.forName("org.apache.hadoop.yarn.client.api.NMClient");
       try {
         Class.forName("org.apache.hadoop.yarn.client.cli.LogsCLI");
-        HADOOP_VERSION.set(HadoopVersions.HADOOP_22);
+        try {
+          Class.forName("org.apache.hadoop.yarn.conf.HAUtil");
+          HADOOP_VERSION.set(HadoopVersions.HADOOP_23);
+        } catch (ClassNotFoundException e) {
+          HADOOP_VERSION.set(HadoopVersions.HADOOP_22);
+        }
       } catch (ClassNotFoundException e) {
         HADOOP_VERSION.set(HadoopVersions.HADOOP_21);
       }


[20/24] twill git commit: (TWILL-180) Reflects YARN application completion status via TwillController

Posted by ch...@apache.org.
(TWILL-180) Reflects YARN application completion status via TwillController

This closes #54 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: cc79f0d0b23c7394dba146868f769291f782122f
Parents: c310b69
Author: Terence Yim <ch...@apache.org>
Authored: Mon Apr 3 23:38:58 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Tue Apr 4 02:09:59 2017 -0700

----------------------------------------------------------------------
 .../org/apache/twill/api/ServiceController.java | 32 +++++++++++++++++++-
 .../AbstractExecutionServiceController.java     | 12 ++++++++
 .../twill/internal/TwillContainerLauncher.java  |  7 +++++
 .../apache/twill/yarn/YarnTwillController.java  | 15 +++++++--
 .../apache/twill/yarn/TaskCompletedTestRun.java | 28 ++++++++++++++++-
 5 files changed, 90 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/cc79f0d0/twill-api/src/main/java/org/apache/twill/api/ServiceController.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/ServiceController.java b/twill-api/src/main/java/org/apache/twill/api/ServiceController.java
index 1ea86b2..bb46290 100644
--- a/twill-api/src/main/java/org/apache/twill/api/ServiceController.java
+++ b/twill-api/src/main/java/org/apache/twill/api/ServiceController.java
@@ -22,6 +22,7 @@ import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import javax.annotation.Nullable;
 
 /**
  * This interface is for controlling a remote running service.
@@ -57,7 +58,8 @@ public interface ServiceController {
    * will be returned.
    *
    * @return a {@link Future} that represents the termination of the service. The future result will be
-   * this {@link ServiceController}. If the service terminated due to exception, the future will carry the exception.
+   * this {@link ServiceController}. If the service terminated with a {@link TerminationStatus#FAILED} status,
+   * calling the {@link Future#get()} on the returning future will throw {@link ExecutionException}.
    */
   Future<? extends ServiceController> terminate();
 
@@ -98,4 +100,32 @@ public interface ServiceController {
    * @throws ExecutionException if the service terminated due to exception.
    */
   void awaitTerminated(long timeout, TimeUnit timeoutUnit) throws TimeoutException, ExecutionException;
+
+  /**
+   * Gets the termination status of the application represented by this controller.
+   *
+   * @return the termination status or {@code null} if the application is still running
+   */
+  @Nullable
+  TerminationStatus getTerminationStatus();
+
+  /**
+   * Enum to represent termination status of the application when it completed.
+   */
+  enum TerminationStatus {
+    /**
+     * Application was completed successfully.
+     */
+    SUCCEEDED,
+
+    /**
+     * Application was killed explicitly.
+     */
+    KILLED,
+
+    /**
+     * Application failed.
+     */
+    FAILED
+  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/cc79f0d0/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
index 580a88f..3ea27fc 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
@@ -36,6 +36,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
 
 /**
  * An abstract base class for implementing {@link ServiceController} that deal with Service state transition and
@@ -47,6 +48,7 @@ public abstract class AbstractExecutionServiceController implements ServiceContr
   private final ListenerExecutors listenerExecutors;
   private final Service serviceDelegate;
   private final SettableFuture<State> terminationFuture;
+  private volatile TerminationStatus terminationStatus;
 
   protected AbstractExecutionServiceController(RunId runId) {
     this.runId = runId;
@@ -87,6 +89,12 @@ public abstract class AbstractExecutionServiceController implements ServiceContr
     });
   }
 
+  @Nullable
+  @Override
+  public TerminationStatus getTerminationStatus() {
+    return terminationStatus;
+  }
+
   @Override
   public void onRunning(final Runnable runnable, Executor executor) {
     addListener(new ServiceListenerAdapter() {
@@ -168,6 +176,10 @@ public abstract class AbstractExecutionServiceController implements ServiceContr
     };
   }
 
+  protected final void setTerminationStatus(TerminationStatus status) {
+    this.terminationStatus = status;
+  }
+
 
   private final class ServiceDelegate extends AbstractIdleService {
     @Override

http://git-wip-us.apache.org/repos/asf/twill/blob/cc79f0d0/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
index 9b6384c..0f8674b 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
@@ -246,6 +246,13 @@ public final class TwillContainerLauncher {
     @Override
     public void completed(int exitStatus) {
       // count down the shutdownLatch to inform any waiting threads that this container is complete
+      if (exitStatus == 0) {
+        setTerminationStatus(TerminationStatus.SUCCEEDED);
+      } else if (exitStatus == 143) {
+        setTerminationStatus(TerminationStatus.KILLED);
+      } else {
+        setTerminationStatus(TerminationStatus.FAILED);
+      }
       shutdownLatch.countDown();
       synchronized (this) {
         forceShutDown();

http://git-wip-us.apache.org/repos/asf/twill/blob/cc79f0d0/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
index 6ea7d8f..335d7ec 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
@@ -166,13 +166,14 @@ final class YarnTwillController extends AbstractTwillController implements Twill
       kill();
     }
 
+    FinalApplicationStatus finalStatus;
     // Poll application status from yarn
     try (ProcessController<YarnApplicationReport> processController = this.processController) {
       Stopwatch stopWatch = new Stopwatch().start();
       long maxTime = TimeUnit.MILLISECONDS.convert(Constants.APPLICATION_MAX_STOP_SECONDS, TimeUnit.SECONDS);
 
       YarnApplicationReport report = processController.getReport();
-      FinalApplicationStatus finalStatus = report.getFinalApplicationStatus();
+      finalStatus = report.getFinalApplicationStatus();
       ApplicationId appId = report.getApplicationId();
       while (finalStatus == FinalApplicationStatus.UNDEFINED &&
           stopWatch.elapsedTime(TimeUnit.MILLISECONDS) < maxTime) {
@@ -180,18 +181,28 @@ final class YarnTwillController extends AbstractTwillController implements Twill
         TimeUnit.SECONDS.sleep(1);
         finalStatus = processController.getReport().getFinalApplicationStatus();
       }
-      LOG.debug("Yarn application {} {} completed with status {}", appName, appId, finalStatus);
 
       // Application not finished after max stop time, kill the application
       if (finalStatus == FinalApplicationStatus.UNDEFINED) {
         kill();
+        finalStatus = FinalApplicationStatus.KILLED;
       }
     } catch (Exception e) {
       LOG.warn("Exception while waiting for application report: {}", e.getMessage(), e);
       kill();
+      finalStatus = FinalApplicationStatus.KILLED;
     }
 
     super.doShutDown();
+
+    if (finalStatus == FinalApplicationStatus.FAILED) {
+      // If we know the app status is failed, throw an exception to make this controller goes into error state.
+      // All other final status are not treated as failure as we can't be sure.
+      setTerminationStatus(TerminationStatus.FAILED);
+      throw new RuntimeException(String.format("Yarn application completed with failure %s, %s.", appName, getRunId()));
+    }
+    setTerminationStatus(finalStatus == FinalApplicationStatus.SUCCEEDED
+                           ? TerminationStatus.SUCCEEDED : TerminationStatus.KILLED);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/twill/blob/cc79f0d0/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
index 51031d4..6fbdc2d 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
@@ -20,6 +20,7 @@ package org.apache.twill.yarn;
 import com.google.common.base.Throwables;
 import org.apache.twill.api.AbstractTwillRunnable;
 import org.apache.twill.api.ResourceSpecification;
+import org.apache.twill.api.ServiceController;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
 import org.apache.twill.api.logging.PrinterLogHandler;
@@ -82,7 +83,32 @@ public final class TaskCompletedTestRun extends BaseYarnTest {
 
     Assert.assertTrue(runLatch.await(1, TimeUnit.MINUTES));
     controller.awaitTerminated(1, TimeUnit.MINUTES);
+    Assert.assertEquals(ServiceController.TerminationStatus.SUCCEEDED, controller.getTerminationStatus());
+  }
+
+  @Test
+  public void testFailureComplete() throws TimeoutException, ExecutionException, InterruptedException {
+    TwillRunner twillRunner = getTwillRunner();
+
+    // Start the app with an invalid ClassLoader. This will cause the AM fails to start.
+    TwillController controller = twillRunner.prepare(new SleepTask(),
+                                                     ResourceSpecification.Builder.with()
+                                                       .setVirtualCores(1)
+                                                       .setMemory(512, ResourceSpecification.SizeUnit.MEGA)
+                                                       .setInstances(1).build())
+      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
+      .setClassLoader("InvalidClassLoader")
+      .start();
+
+    final CountDownLatch terminateLatch = new CountDownLatch(1);
+    controller.onTerminated(new Runnable() {
+      @Override
+      public void run() {
+        terminateLatch.countDown();
+      }
+    }, Threads.SAME_THREAD_EXECUTOR);
 
-    TimeUnit.SECONDS.sleep(2);
+    Assert.assertTrue(terminateLatch.await(2, TimeUnit.MINUTES));
+    Assert.assertEquals(ServiceController.TerminationStatus.FAILED, controller.getTerminationStatus());
   }
 }


[11/24] twill git commit: (TWILL-226) Remove the deprecated HDFSLocationFactory

Posted by ch...@apache.org.
(TWILL-226) Remove the deprecated HDFSLocationFactory

This closes #44 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/7f348716
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/7f348716
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/7f348716

Branch: refs/heads/site
Commit: 7f3487169da9639e83f797f38733c8c5d5d4d9f6
Parents: 390dfab
Author: Terence Yim <ch...@apache.org>
Authored: Mon Mar 27 16:14:17 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Mar 27 16:54:02 2017 -0700

----------------------------------------------------------------------
 .../apache/twill/filesystem/HDFSLocation.java   | 371 -------------------
 .../twill/filesystem/HDFSLocationFactory.java   | 107 ------
 .../apache/twill/internal/yarn/YarnUtils.java   |   6 +-
 .../twill/filesystem/HDFSLocationTest.java      |  31 --
 4 files changed, 1 insertion(+), 514 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/7f348716/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java b/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
deleted file mode 100644
index ef1138e..0000000
--- a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
+++ /dev/null
@@ -1,371 +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.
- */
-package org.apache.twill.filesystem;
-
-import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableList;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.security.AccessControlException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.UUID;
-
-/**
- * A concrete implementation of {@link Location} for the HDFS filesystem using {@link FileSystem}.
- */
-final class HDFSLocation implements Location {
-  private final FileSystem fs;
-  private final Path path;
-  private final HDFSLocationFactory locationFactory;
-
-  /**
-   * Constructs a HDFSLocation.
-   *
-   * @param locationFactory The {@link HDFSLocationFactory} that creates this instance.
-   * @param path of the file.
-   */
-  HDFSLocation(HDFSLocationFactory locationFactory, Path path) {
-    this.fs = locationFactory.getFileSystem();
-    this.path = path;
-    this.locationFactory = locationFactory;
-  }
-
-  /**
-   * Checks if this location exists on HDFS.
-   *
-   * @return true if found; false otherwise.
-   * @throws IOException
-   */
-  @Override
-  public boolean exists() throws IOException {
-    return fs.exists(path);
-  }
-
-  /**
-   * @return An {@link InputStream} for this location on HDFS.
-   * @throws IOException
-   */
-  @Override
-  public InputStream getInputStream() throws IOException {
-    return fs.open(path);
-  }
-
-  /**
-   * @return An {@link OutputStream} for this location on HDFS.
-   * @throws IOException
-   */
-  @Override
-  public OutputStream getOutputStream() throws IOException {
-    return fs.create(path);
-  }
-
-  @Override
-  public OutputStream getOutputStream(String permission) throws IOException {
-    FsPermission fsPermission = parsePermissions(permission);
-    OutputStream os = fs.create(path,
-                                fsPermission,
-                                true,
-                                fs.getConf().getInt("io.file.buffer.size", 4096),
-                                fs.getDefaultReplication(path),
-                                fs.getDefaultBlockSize(path),
-                                null);
-    // Set the permission explicitly again to skip the umask
-    fs.setPermission(path, fsPermission);
-    return os;
-  }
-
-  /**
-   * Appends the child to the current {@link Location} on HDFS.
-   * <p>
-   * Returns a new instance of Location.
-   * </p>
-   *
-   * @param child to be appended to this location.
-   * @return A new instance of {@link Location}
-   * @throws IOException
-   */
-  @Override
-  public Location append(String child) throws IOException {
-    if (child.startsWith("/")) {
-      child = child.substring(1);
-    }
-    return new HDFSLocation(locationFactory, new Path(URI.create(path.toUri() + "/" + child)));
-  }
-
-  @Override
-  public Location getTempFile(String suffix) throws IOException {
-    Path path = new Path(
-      URI.create(this.path.toUri() + "." + UUID.randomUUID() + (suffix == null ? TEMP_FILE_SUFFIX : suffix)));
-    return new HDFSLocation(locationFactory, path);
-  }
-
-  /**
-   * @return Returns the name of the file or directory denoted by this abstract pathname.
-   */
-  @Override
-  public String getName() {
-    return path.getName();
-  }
-
-  @Override
-  public boolean createNew() throws IOException {
-    return fs.createNewFile(path);
-  }
-
-  @Override
-  public boolean createNew(String permission) throws IOException {
-    try {
-      FsPermission fsPermission = parsePermissions(permission);
-      fs.create(path, fsPermission, EnumSet.of(CreateFlag.CREATE),
-                fs.getConf().getInt("io.file.buffer.size", 4096),
-                fs.getDefaultReplication(path),
-                fs.getDefaultBlockSize(path),
-                null).close();
-      // Set the permission explicitly again to skip the umask
-      fs.setPermission(path, fsPermission);
-      return true;
-    } catch (FileAlreadyExistsException e) {
-      return false;
-    }
-  }
-
-  @Override
-  public String getOwner() throws IOException {
-    return fs.getFileStatus(path).getOwner();
-  }
-
-  @Override
-  public String getGroup() throws IOException {
-    return fs.getFileStatus(path).getGroup();
-  }
-
-  @Override
-  public void setGroup(String group) throws IOException {
-    fs.setOwner(path, null, group);
-  }
-
-  @Override
-  public String getPermissions() throws IOException {
-    FsPermission permission = fs.getFileStatus(path).getPermission();
-    return permission.getUserAction().SYMBOL + permission.getGroupAction().SYMBOL + permission.getOtherAction().SYMBOL;
-  }
-
-  @Override
-  public void setPermissions(String permission) throws IOException {
-    fs.setPermission(path, parsePermissions(permission));
-  }
-
-  /**
-   * @return A {@link URI} for this location on HDFS.
-   */
-  @Override
-  public URI toURI() {
-    return path.toUri();
-  }
-
-  /**
-   * Deletes the file or directory denoted by this abstract pathname. If this
-   * pathname denotes a directory, then the directory must be empty in order
-   * to be deleted.
-   *
-   * @return true if and only if the file or directory is successfully deleted; false otherwise.
-   */
-  @Override
-  public boolean delete() throws IOException {
-    return fs.delete(path, false);
-  }
-
-  @Override
-  public boolean delete(boolean recursive) throws IOException {
-    return fs.delete(path, true);
-  }
-
-  @Override
-  public Location renameTo(Location destination) throws IOException {
-    // Destination will always be of the same type as this location.
-    if (fs instanceof DistributedFileSystem) {
-      ((DistributedFileSystem) fs).rename(path, ((HDFSLocation) destination).path, Options.Rename.OVERWRITE);
-      return new HDFSLocation(locationFactory, new Path(destination.toURI()));
-    }
-
-    if (fs.rename(path, ((HDFSLocation) destination).path)) {
-      return new HDFSLocation(locationFactory, new Path(destination.toURI()));
-    } else {
-      return null;
-    }
-  }
-
-  @Override
-  public boolean mkdirs() throws IOException {
-    try {
-      if (fs.exists(path)) {
-        return false;
-      }
-      return fs.mkdirs(path);
-    } catch (FileAlreadyExistsException | AccessControlException e) {
-      // curiously, if one of the parent dirs exists but is a file, Hadoop throws this:
-      // org.apache...AccessControlException: Permission denied: user=..., access=EXECUTE, inode=".../existingfile"
-      // however, if the directory itself exists, it will throw FileAlreadyExistsException
-      return false;
-    }
-  }
-
-  @Override
-  public boolean mkdirs(String permission) throws IOException {
-    return mkdirs(path, parsePermissions(permission));
-  }
-
-  /**
-   * Helper to create a directory and its parents id necessary, all with the given permissions.
-   * We cannot use the fs.mkdirs() because that would apply the umask to the permissions.
-   */
-  private boolean mkdirs(Path path, FsPermission permission) throws IOException {
-    try {
-      if (fs.exists(path)) {
-        return false;
-      }
-    } catch (AccessControlException e) {
-      // curiously, if one of the parent dirs exists but is a file, Hadoop throws this:
-      // org.apache...AccessControlException: Permission denied: user=..., access=EXECUTE, inode=".../existingfile"
-      return false;
-    }
-    Path parent = path.getParent();
-    if (null == parent) {
-      return false;
-    }
-    // if parent exists, attempt to create the path as a directory.
-    if (fs.exists(parent)) {
-      return mkdir(path, permission);
-    }
-    // attempt to create the parent with the proper permissions
-    if (!mkdirs(parent, permission) && !fs.isDirectory(parent)) {
-      return false;
-    }
-    // now the parent exists and we can create this directory
-    return mkdir(path, permission);
-  }
-
-  /**
-   * Helper to create a directory (but not its parents) with the given permissions.
-   * We cannot use fs.mkdirs() and then apply the permissions to override the umask.
-   */
-  private boolean mkdir(Path path, FsPermission permission) throws IOException {
-    try {
-      if (!fs.mkdirs(path) && !fs.isDirectory(path)) {
-        return false;
-      }
-    } catch (FileAlreadyExistsException e) {
-      return false;
-    }
-    // explicitly set permissions to get around the umask
-    fs.setPermission(path, permission);
-    return true;
-  }
-
-  /**
-   * @return Length of file.
-   */
-  @Override
-  public long length() throws IOException {
-    return fs.getFileStatus(path).getLen();
-  }
-
-  @Override
-  public long lastModified() throws IOException {
-    return fs.getFileStatus(path).getModificationTime();
-  }
-
-  @Override
-  public boolean isDirectory() throws IOException {
-    return fs.isDirectory(path);
-  }
-
-  @Override
-  public List<Location> list() throws IOException {
-    FileStatus[] statuses = fs.listStatus(path);
-    ImmutableList.Builder<Location> result = ImmutableList.builder();
-    if (statuses != null) {
-      for (FileStatus status : statuses) {
-        if (!Objects.equal(path, status.getPath())) {
-          result.add(new HDFSLocation(locationFactory, status.getPath()));
-        }
-      }
-    }
-    return result.build();
-  }
-
-  @Override
-  public LocationFactory getLocationFactory() {
-    return locationFactory;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    HDFSLocation other = (HDFSLocation) o;
-    return Objects.equal(path, other.path);
-  }
-
-  @Override
-  public int hashCode() {
-    return path.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return path.toString();
-  }
-
-  /**
-   * Parses the given permission to {@link FsPermission}. Since the {@link HDFSLocationFactory} and this class are
-   * deprecated, this method is copied from {@link FileContextLocation} instead of creating an extra library to share.
-   *
-   * @param permission the permission as passed to the {@link #createNew(String)} or {@link #getOutputStream(String)}
-   *                   methods.
-   * @return a new {@link FsPermission}.
-   */
-  private FsPermission parsePermissions(String permission) {
-    if (permission.length() == 3) {
-      return new FsPermission(permission);
-    } else if (permission.length() == 9) {
-      // The FsPermission expect a 10 characters string, which it will ignore the first character
-      return FsPermission.valueOf("-" + permission);
-    } else {
-      throw new IllegalArgumentException("Invalid permission " + permission +
-                                           ". Permission should either be a three digit or nine character string.");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/twill/blob/7f348716/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocationFactory.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocationFactory.java b/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocationFactory.java
deleted file mode 100644
index 728de32..0000000
--- a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocationFactory.java
+++ /dev/null
@@ -1,107 +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.
- */
-package org.apache.twill.filesystem;
-
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Objects;
-
-/**
- * A {@link LocationFactory} that creates HDFS {@link Location} using {@link FileSystem}.
- *
- * @deprecated Deprecated since 0.7.0. Use {@link FileContextLocationFactory} instead.
- */
-@Deprecated
-public final class HDFSLocationFactory implements LocationFactory {
-
-  private final FileSystem fileSystem;
-  private final String pathBase;
-
-  public HDFSLocationFactory(Configuration configuration) {
-    this(getFileSystem(configuration));
-  }
-  
-  public HDFSLocationFactory(Configuration configuration, String pathBase) {
-    this(getFileSystem(configuration), pathBase);
-  }
-
-  public HDFSLocationFactory(FileSystem fileSystem) {
-    this(fileSystem, "/");
-  }
-
-  public HDFSLocationFactory(FileSystem fileSystem, String pathBase) {
-    String base = pathBase.equals("/") ? "" : pathBase;
-    base = base.endsWith("/") ? base.substring(0, base.length() - 1) : base;
-
-    this.fileSystem = fileSystem;
-    this.pathBase = base;
-  }
-
-  @Override
-  public Location create(String path) {
-    if (path.startsWith("/")) {
-      path = path.substring(1);
-    }
-    return new HDFSLocation(this, new Path(fileSystem.getUri() + "/" + pathBase + "/" + path));
-  }
-
-  @Override
-  public Location create(URI uri) {
-    URI fsURI = fileSystem.getUri();
-    if (Objects.equals(fsURI.getScheme(), uri.getScheme())
-      && Objects.equals(fsURI.getAuthority(), uri.getAuthority())) {
-      // It's a full URI
-      return new HDFSLocation(this, new Path(uri));
-    }
-
-    if (uri.isAbsolute()) {
-      // Needs to be of the same scheme
-      Preconditions.checkArgument(Objects.equals(fsURI.getScheme(), uri.getScheme()),
-                                  "Only URI with '%s' scheme is supported", fsURI.getScheme());
-      return new HDFSLocation(this, new Path(fileSystem.getUri() + uri.getPath()));
-    }
-
-    return create(uri.getPath());
-  }
-
-  @Override
-  public Location getHomeLocation() {
-    return new HDFSLocation(this, fileSystem.getHomeDirectory());
-  }
-
-  /**
-   * Returns the underlying {@link FileSystem} object.
-   */
-  public FileSystem getFileSystem() {
-    return fileSystem;
-  }
-
-  private static FileSystem getFileSystem(Configuration configuration) {
-    try {
-      return FileSystem.get(configuration);
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/twill/blob/7f348716/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
index e931144..3c4c270 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.util.Records;
 import org.apache.twill.api.LocalFile;
 import org.apache.twill.filesystem.FileContextLocationFactory;
 import org.apache.twill.filesystem.ForwardingLocationFactory;
-import org.apache.twill.filesystem.HDFSLocationFactory;
 import org.apache.twill.filesystem.LocationFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -163,7 +162,7 @@ public class YarnUtils {
     FileSystem fileSystem = getFileSystem(locationFactory, config);
 
     if (fileSystem == null) {
-      LOG.warn("Unexpected: LocationFactory is neither FileContextLocationFactory nor HDFSLocationFactory.");
+      LOG.warn("Unexpected: LocationFactory is not backed by FileContextLocationFactory");
       return ImmutableList.of();
     }
 
@@ -320,9 +319,6 @@ public class YarnUtils {
    */
   @Nullable
   private static FileSystem getFileSystem(LocationFactory locationFactory, Configuration config) throws IOException {
-    if (locationFactory instanceof HDFSLocationFactory) {
-      return ((HDFSLocationFactory) locationFactory).getFileSystem();
-    }
     if (locationFactory instanceof ForwardingLocationFactory) {
       return getFileSystem(((ForwardingLocationFactory) locationFactory).getDelegate(), config);
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/7f348716/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
deleted file mode 100644
index de0b4c5..0000000
--- a/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
+++ /dev/null
@@ -1,31 +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.
- */
-package org.apache.twill.filesystem;
-
-import java.io.IOException;
-
-/**
- * Everything here is set up the same as for FileContextLocation; except that we use an HDFSLocationFactory.
- */
-public class HDFSLocationTest extends FileContextLocationTest {
-
-  @Override
-  protected LocationFactory doCreateLocationFactory(String pathBase) throws IOException {
-    return new HDFSLocationFactory(dfsCluster.getFileSystem(), pathBase);
-  }
-}


[24/24] twill git commit: Merge branch 'master' into site

Posted by ch...@apache.org.
Merge branch 'master' into site


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/3680e00f
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/3680e00f
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/3680e00f

Branch: refs/heads/site
Commit: 3680e00f0f60700eb0b4ffff94e6fe8f34c77d6c
Parents: 7bdf857 401496a
Author: Terence Yim <ch...@apache.org>
Authored: Sat Apr 8 11:08:16 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Sat Apr 8 11:08:16 2017 -0700

----------------------------------------------------------------------
 NOTICE                                          |   2 +-
 README.md                                       |   4 +-
 pom.xml                                         |   6 +-
 twill-api/pom.xml                               |   2 +-
 .../main/java/org/apache/twill/api/Configs.java |  42 +++
 .../apache/twill/api/SecureStoreUpdater.java    |   5 +
 .../org/apache/twill/api/ServiceController.java |  32 +-
 .../org/apache/twill/api/TwillPreparer.java     |  21 ++
 .../java/org/apache/twill/api/TwillRunner.java  |  20 +
 .../twill/api/security/SecureStoreRenewer.java  |  41 ++
 .../twill/api/security/SecureStoreWriter.java   |  36 ++
 twill-common/pom.xml                            |   2 +-
 .../org/apache/twill/internal/Constants.java    |  12 +-
 twill-core/pom.xml                              |   2 +-
 .../AbstractExecutionServiceController.java     |  12 +
 .../twill/internal/AbstractTwillController.java |  36 +-
 .../java/org/apache/twill/internal/EnvKeys.java |   2 -
 .../internal/TwillContainerController.java      |  10 +-
 .../twill/internal/TwillContainerLauncher.java  |  12 +
 .../internal/TwillRuntimeSpecification.java     |  26 +-
 .../internal/json/ResourceReportAdapter.java    |   1 +
 .../json/TwillRuntimeSpecificationCodec.java    |   8 +-
 .../twill/internal/logging/KafkaAppender.java   |  30 +-
 .../apache/twill/launcher/TwillLauncher.java    |  44 ++-
 .../apache/twill/internal/ControllerTest.java   |  11 +-
 twill-discovery-api/pom.xml                     |   2 +-
 twill-discovery-core/pom.xml                    |   2 +-
 twill-examples/echo/pom.xml                     |   2 +-
 twill-examples/pom.xml                          |   2 +-
 twill-examples/yarn/pom.xml                     |   2 +-
 twill-ext/pom.xml                               |   2 +-
 twill-java8-test/pom.xml                        |   2 +-
 twill-yarn/pom.xml                              |   2 +-
 .../internal/yarn/Hadoop20YarnAppClient.java    | 120 +++---
 .../yarn/Hadoop20YarnApplicationReport.java     |   4 +-
 .../internal/yarn/Hadoop21YarnAppClient.java    | 133 ++++---
 .../internal/yarn/Hadoop23YarnAppClient.java    | 111 ++++++
 .../filesystem/FileContextLocationFactory.java  |  77 +++-
 .../apache/twill/filesystem/HDFSLocation.java   | 371 -------------------
 .../twill/filesystem/HDFSLocationFactory.java   | 107 ------
 .../org/apache/twill/internal/ServiceMain.java  | 125 +++----
 .../ApplicationMasterLiveNodeData.java          |  15 +-
 .../appmaster/ApplicationMasterMain.java        |  96 +++--
 .../appmaster/ApplicationMasterService.java     | 218 +++++------
 .../internal/appmaster/ExpectedContainers.java  |  13 +-
 .../appmaster/PlacementPolicyManager.java       |  77 ++--
 .../appmaster/RunnableContainerRequest.java     |   6 +-
 .../internal/appmaster/RunningContainers.java   |  11 +-
 .../internal/appmaster/TrackerService.java      |  37 +-
 .../internal/container/TwillContainerMain.java  |  36 +-
 .../container/TwillContainerService.java        |   7 +-
 .../internal/yarn/AbstractYarnAMClient.java     |  50 +--
 .../internal/yarn/AbstractYarnTwillService.java |  18 +-
 .../yarn/VersionDetectYarnAMClientFactory.java  |   2 +-
 .../yarn/VersionDetectYarnAppClientFactory.java |  25 +-
 .../twill/internal/yarn/YarnAMClient.java       |  40 +-
 .../apache/twill/internal/yarn/YarnUtils.java   | 100 +++--
 .../twill/yarn/LocationSecureStoreRenewer.java  |  49 +++
 .../twill/yarn/LocationSecureStoreUpdater.java  |  54 ---
 .../apache/twill/yarn/ResourceReportClient.java |  28 +-
 .../apache/twill/yarn/YarnTwillController.java  |  79 +++-
 .../twill/yarn/YarnTwillControllerFactory.java  |   2 +-
 .../apache/twill/yarn/YarnTwillPreparer.java    | 269 ++++++++------
 .../twill/yarn/YarnTwillRunnerService.java      | 312 +++++++++++-----
 .../filesystem/FileContextLocationTest.java     |  58 +--
 .../twill/filesystem/HDFSLocationTest.java      |  31 --
 .../twill/filesystem/LocalLocationTest.java     |  23 +-
 .../twill/filesystem/LocationTestBase.java      | 367 +++++++++---------
 .../apache/twill/yarn/ContainerSizeTestRun.java |  10 +
 .../apache/twill/yarn/CustomClassLoader.java    |  87 +++++
 .../twill/yarn/CustomClassLoaderRunnable.java   |  56 +++
 .../twill/yarn/CustomClassLoaderTestRun.java    |  42 +++
 .../apache/twill/yarn/LogHandlerTestRun.java    |  44 +++
 .../org/apache/twill/yarn/LogLevelTestRun.java  |  26 +-
 .../twill/yarn/ResourceReportTestRun.java       |  47 ++-
 .../apache/twill/yarn/TaskCompletedTestRun.java |  28 +-
 .../java/org/apache/twill/yarn/TwillTester.java |   5 +-
 .../org/apache/twill/yarn/YarnTestSuite.java    |   1 +
 twill-yarn/src/test/resources/logback-test.xml  |   1 +
 twill-zookeeper/pom.xml                         |   2 +-
 80 files changed, 2178 insertions(+), 1677 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/3680e00f/pom.xml
----------------------------------------------------------------------


[18/24] twill git commit: (TWILL-176) Redirect all requests to tracker URL to /resources path

Posted by ch...@apache.org.
(TWILL-176) Redirect all requests to tracker URL to /resources path

- "/resources" is the only path supported for the tracker service for now

This fixes #52 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/10fff16c
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/10fff16c
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/10fff16c

Branch: refs/heads/site
Commit: 10fff16c07ca7e47f413b1965e72d5544e712433
Parents: c8e2a61
Author: Terence Yim <ch...@apache.org>
Authored: Mon Apr 3 14:42:44 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Apr 3 15:10:45 2017 -0700

----------------------------------------------------------------------
 .../internal/json/ResourceReportAdapter.java    |  1 +
 .../internal/appmaster/TrackerService.java      | 37 +++++++++++---------
 2 files changed, 22 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/10fff16c/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
index 4427704..7484f7a 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
@@ -39,6 +39,7 @@ public final class ResourceReportAdapter {
   private ResourceReportAdapter() {
     gson = new GsonBuilder()
               .serializeNulls()
+              .setPrettyPrinting()
               .registerTypeAdapter(TwillRunResources.class, new TwillRunResourcesCodec())
               .registerTypeAdapter(ResourceReport.class, new ResourceReportCodec())
               .create();

http://git-wip-us.apache.org/repos/asf/twill/blob/10fff16c/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
index d3f7c86..f91efcc 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
@@ -61,6 +61,7 @@ import java.io.Writer;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -178,33 +179,33 @@ public final class TrackerService extends AbstractIdleService {
   final class ReportHandler extends SimpleChannelUpstreamHandler {
     private final ResourceReportAdapter reportAdapter;
 
-    public ReportHandler() {
+    ReportHandler() {
       this.reportAdapter = ResourceReportAdapter.create();
     }
 
     @Override
     public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
       HttpRequest request = (HttpRequest) e.getMessage();
-      if (!isValid(request)) {
-        write404(e);
+      if (request.getMethod() != HttpMethod.GET) {
+        HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.METHOD_NOT_ALLOWED);
+        response.setHeader(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=UTF-8");
+        response.setContent(ChannelBuffers.wrappedBuffer("Only GET is supported".getBytes(StandardCharsets.UTF_8)));
+        writeResponse(e.getChannel(), response);
         return;
       }
 
-      writeResponse(e);
-    }
-
-    // only accepts GET on /resources for now
-    private boolean isValid(HttpRequest request) {
-      return (request.getMethod() == HttpMethod.GET) && PATH.equals(request.getUri());
-    }
+      if (!PATH.equals(request.getUri())) {
+        // Redirect all GET call to the /resources path.
+        HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.TEMPORARY_REDIRECT);
+        response.setHeader(HttpHeaders.Names.LOCATION, PATH);
+        writeResponse(e.getChannel(), response);
+        return;
+      }
 
-    private void write404(MessageEvent e) {
-      HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NOT_FOUND);
-      ChannelFuture future = e.getChannel().write(response);
-      future.addListener(ChannelFutureListener.CLOSE);
+      writeResourceReport(e.getChannel());
     }
 
-    private void writeResponse(MessageEvent e) {
+    private void writeResourceReport(Channel channel) {
       HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
       response.setHeader(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=UTF-8");
 
@@ -217,7 +218,11 @@ public final class TrackerService extends AbstractIdleService {
         LOG.error("error writing resource report", e1);
       }
       response.setContent(content);
-      ChannelFuture future = e.getChannel().write(response);
+      writeResponse(channel, response);
+    }
+
+    private void writeResponse(Channel channel, HttpResponse response) {
+      ChannelFuture future = channel.write(response);
       future.addListener(ChannelFutureListener.CLOSE);
     }
 


[06/24] twill git commit: (TWILL-225) Allow configurations overridable per TwillPreprer

Posted by ch...@apache.org.
(TWILL-225) Allow configurations overridable per TwillPreprer

- Also increased the vmen-pmen ration in TwillTester to avoid test failure due to reduced container size
  - There is no way to disable vmen-pmen ratio check in Hadoop 2.0.

This closes #39 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: e154bfedb93ec8055def1bf55a4d2d8bc8ccf833
Parents: c4cceef
Author: Terence Yim <ch...@apache.org>
Authored: Sat Mar 18 18:31:57 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Tue Mar 21 13:32:12 2017 -0700

----------------------------------------------------------------------
 .../org/apache/twill/api/TwillPreparer.java     |   8 ++
 .../apache/twill/yarn/YarnTwillPreparer.java    | 103 +++++++++++++------
 .../twill/yarn/YarnTwillRunnerService.java      |   9 +-
 .../apache/twill/yarn/ContainerSizeTestRun.java |  10 ++
 .../java/org/apache/twill/yarn/TwillTester.java |   2 +-
 5 files changed, 96 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/e154bfed/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java b/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
index fd568b3..43b751b 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
@@ -30,6 +30,14 @@ import java.util.concurrent.TimeUnit;
 public interface TwillPreparer {
 
   /**
+   * Overrides the default configuration with the given set of configurations.
+   *
+   * @param config set of configurations to override
+   * @return This {@link TwillPreparer}
+   */
+  TwillPreparer withConfiguration(Map<String, String> config);
+
+  /**
    * Adds a {@link LogHandler} for receiving an application log.
    * @param handler The {@link LogHandler}.
    * @return This {@link TwillPreparer}.

http://git-wip-us.apache.org/repos/asf/twill/blob/e154bfed/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index de03a7a..2d1edd0 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -40,6 +40,7 @@ import com.google.common.reflect.TypeToken;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import joptsimple.OptionSpec;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -80,6 +81,7 @@ import org.apache.twill.internal.json.TwillRuntimeSpecificationAdapter;
 import org.apache.twill.internal.utils.Dependencies;
 import org.apache.twill.internal.utils.Paths;
 import org.apache.twill.internal.utils.Resources;
+import org.apache.twill.internal.yarn.VersionDetectYarnAppClientFactory;
 import org.apache.twill.internal.yarn.YarnAppClient;
 import org.apache.twill.internal.yarn.YarnApplicationReport;
 import org.apache.twill.internal.yarn.YarnUtils;
@@ -126,9 +128,8 @@ final class YarnTwillPreparer implements TwillPreparer {
     }
   };
 
-  private final YarnConfiguration yarnConfig;
+  private final Configuration config;
   private final TwillSpecification twillSpec;
-  private final YarnAppClient yarnAppClient;
   private final String zkConnectString;
   private final Location appLocation;
   private final YarnTwillControllerFactory controllerFactory;
@@ -143,9 +144,6 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final Map<String, Map<String, String>> environments = Maps.newHashMap();
   private final List<String> applicationClassPaths = Lists.newArrayList();
   private final Credentials credentials;
-  private final int reservedMemory;
-  private final double minHeapRatio;
-  private final File localStagingDir;
   private final Map<String, Map<String, String>> logLevels = Maps.newHashMap();
   private final LocationCache locationCache;
   private final Set<URL> twillClassPaths;
@@ -155,25 +153,16 @@ final class YarnTwillPreparer implements TwillPreparer {
   private ClassAcceptor classAcceptor;
   private final Map<String, Integer> maxRetries = Maps.newHashMap();
 
-  YarnTwillPreparer(YarnConfiguration yarnConfig, TwillSpecification twillSpec, RunId runId,
-                    YarnAppClient yarnAppClient, String zkConnectString, Location appLocation, Set<URL> twillClassPaths,
+  YarnTwillPreparer(Configuration config, TwillSpecification twillSpec, RunId runId,
+                    String zkConnectString, Location appLocation, Set<URL> twillClassPaths,
                     String extraOptions, LocationCache locationCache, YarnTwillControllerFactory controllerFactory) {
-    this.yarnConfig = yarnConfig;
+    this.config = config;
     this.twillSpec = twillSpec;
     this.runId = runId;
-    this.yarnAppClient = yarnAppClient;
     this.zkConnectString = zkConnectString;
     this.appLocation = appLocation;
     this.controllerFactory = controllerFactory;
     this.credentials = createCredentials();
-    this.reservedMemory = yarnConfig.getInt(Configs.Keys.JAVA_RESERVED_MEMORY_MB,
-                                            Configs.Defaults.JAVA_RESERVED_MEMORY_MB);
-    // doing this way to support hadoop-2.0 profile
-    String minHeapRatioStr = yarnConfig.get(Configs.Keys.HEAP_RESERVED_MIN_RATIO);
-    this.minHeapRatio = (minHeapRatioStr == null) ?
-            Configs.Defaults.HEAP_RESERVED_MIN_RATIO : Double.parseDouble(minHeapRatioStr);
-    this.localStagingDir = new File(yarnConfig.get(Configs.Keys.LOCAL_STAGING_DIRECTORY,
-                                                   Configs.Defaults.LOCAL_STAGING_DIRECTORY));
     this.extraOptions = extraOptions;
     this.classAcceptor = new ClassAcceptor();
     this.locationCache = locationCache;
@@ -187,6 +176,14 @@ final class YarnTwillPreparer implements TwillPreparer {
   }
 
   @Override
+  public TwillPreparer withConfiguration(Map<String, String> config) {
+    for (Map.Entry<String, String> entry : config.entrySet()) {
+      this.config.set(entry.getKey(), entry.getValue());
+    }
+    return this;
+  }
+
+  @Override
   public TwillPreparer addLogHandler(LogHandler handler) {
     logHandlers.add(handler);
     return this;
@@ -362,6 +359,7 @@ final class YarnTwillPreparer implements TwillPreparer {
   @Override
   public TwillController start(long timeout, TimeUnit timeoutUnit) {
     try {
+      final YarnAppClient yarnAppClient = new VersionDetectYarnAppClientFactory().create(config);
       final ProcessLauncher<ApplicationMasterInfo> launcher = yarnAppClient.createLauncher(twillSpec, schedulerQueue);
       final ApplicationMasterInfo appMasterInfo = launcher.getContainerInfo();
       Callable<ProcessController<YarnApplicationReport>> submitTask =
@@ -373,11 +371,11 @@ final class YarnTwillPreparer implements TwillPreparer {
             Map<String, LocalFile> localFiles = Maps.newHashMap();
 
             createLauncherJar(localFiles);
-            createTwillJar(createBundler(classAcceptor), localFiles);
+            createTwillJar(createBundler(classAcceptor), yarnAppClient, localFiles);
             createApplicationJar(createApplicationJarBundler(classAcceptor), localFiles);
             createResourcesJar(createBundler(classAcceptor), localFiles);
 
-            Path runtimeConfigDir = Files.createTempDirectory(localStagingDir.toPath(),
+            Path runtimeConfigDir = Files.createTempDirectory(getLocalStagingDir().toPath(),
                                                               Constants.Files.RUNTIME_CONFIG_JAR);
             try {
               saveSpecification(twillSpec, runtimeConfigDir.resolve(Constants.Files.TWILL_SPEC));
@@ -401,12 +399,11 @@ final class YarnTwillPreparer implements TwillPreparer {
             //     org.apache.twill.internal.appmaster.ApplicationMasterMain
             //     false
 
-            int reservedMemoryMB = yarnConfig.getInt(Configs.Keys.YARN_AM_RESERVED_MEMORY_MB,
-                                                     Configs.Defaults.YARN_AM_RESERVED_MEMORY_MB);
-            int memory = Resources.computeMaxHeapSize(appMasterInfo.getMemoryMB(),
-                                                      reservedMemoryMB,
-                                                      minHeapRatio);
-            return launcher.prepareLaunch(ImmutableMap.<String, String>of(), localFiles.values(), credentials)
+            int reservedMemoryMB = config.getInt(Configs.Keys.YARN_AM_RESERVED_MEMORY_MB,
+                                                 Configs.Defaults.YARN_AM_RESERVED_MEMORY_MB);
+            int memory = Resources.computeMaxHeapSize(appMasterInfo.getMemoryMB(), reservedMemoryMB, getMinHeapRatio());
+            return launcher.prepareLaunch(ImmutableMap.<String, String>of(), localFiles.values(),
+                                          createSubmissionCredentials())
               .addCommand(
                 "$JAVA_HOME/bin/java",
                 "-Djava.io.tmpdir=tmp",
@@ -431,6 +428,29 @@ final class YarnTwillPreparer implements TwillPreparer {
     }
   }
 
+  /**
+   * Returns the minimum heap ratio based on the configuration.
+   */
+  private double getMinHeapRatio() {
+    // doing this way to support hadoop-2.0 profile
+    String minHeapRatioStr = config.get(Configs.Keys.HEAP_RESERVED_MIN_RATIO);
+    return (minHeapRatioStr == null) ? Configs.Defaults.HEAP_RESERVED_MIN_RATIO : Double.parseDouble(minHeapRatioStr);
+  }
+
+  /**
+   * Returns the reserved memory size in MB based on the configuration.
+   */
+  private int getReservedMemory() {
+    return config.getInt(Configs.Keys.JAVA_RESERVED_MEMORY_MB, Configs.Defaults.JAVA_RESERVED_MEMORY_MB);
+  }
+
+  /**
+   * Returns the local staging directory based on the configuration.
+   */
+  private File getLocalStagingDir() {
+    return new File(config.get(Configs.Keys.LOCAL_STAGING_DIRECTORY, Configs.Defaults.LOCAL_STAGING_DIRECTORY));
+  }
+
   private void setEnv(String runnableName, Map<String, String> env, boolean overwrite) {
     Map<String, String> environment = environments.get(runnableName);
     if (environment == null) {
@@ -455,21 +475,40 @@ final class YarnTwillPreparer implements TwillPreparer {
     this.logLevels.put(runnableName, newLevels);
   }
 
+  /**
+   * Creates an {@link Credentials} by copying the {@link Credentials} of the current user.
+   */
   private Credentials createCredentials() {
     Credentials credentials = new Credentials();
 
     try {
       credentials.addAll(UserGroupInformation.getCurrentUser().getCredentials());
+    } catch (IOException e) {
+      LOG.warn("Failed to get current user UGI. Current user credentials not added.", e);
+    }
+    return credentials;
+  }
 
-      List<Token<?>> tokens = YarnUtils.addDelegationTokens(yarnConfig, appLocation.getLocationFactory(), credentials);
+  /**
+   * Creates a {@link Credentials} for the application submission.
+   */
+  private Credentials createSubmissionCredentials() {
+    Credentials credentials = new Credentials();
+    try {
+      // Acquires delegation token for the location
+      List<Token<?>> tokens = YarnUtils.addDelegationTokens(config, appLocation.getLocationFactory(), credentials);
       if (LOG.isDebugEnabled()) {
         for (Token<?> token : tokens) {
           LOG.debug("Delegation token acquired for {}, {}", appLocation, token);
         }
       }
     } catch (IOException e) {
-      LOG.warn("Failed to check for secure login type. Not gathering any delegation token.", e);
+      LOG.warn("Failed to acquire delegation token for location {}", appLocation);
     }
+
+    // Copy the user provided credentials.
+    // It will override the location delegation tokens acquired above if user supplies it.
+    credentials.addAll(this.credentials);
     return credentials;
   }
 
@@ -481,7 +520,9 @@ final class YarnTwillPreparer implements TwillPreparer {
     return new DefaultLocalFile(name, location.toURI(), location.lastModified(), location.length(), archive, null);
   }
 
-  private void createTwillJar(final ApplicationBundler bundler, Map<String, LocalFile> localFiles) throws IOException {
+  private void createTwillJar(final ApplicationBundler bundler,
+                              final YarnAppClient yarnAppClient,
+                              Map<String, LocalFile> localFiles) throws IOException {
     LOG.debug("Create and copy {}", Constants.Files.TWILL_JAR);
     Location location = locationCache.get(Constants.Files.TWILL_JAR, new LocationCache.Loader() {
       @Override
@@ -633,8 +674,8 @@ final class YarnTwillPreparer implements TwillPreparer {
       TwillRuntimeSpecificationAdapter.create().toJson(
         new TwillRuntimeSpecification(newTwillSpec, appLocation.getLocationFactory().getHomeLocation().getName(),
                                       appLocation.toURI(), zkConnectString, runId, twillSpec.getName(),
-                                      reservedMemory, yarnConfig.get(YarnConfiguration.RM_SCHEDULER_ADDRESS),
-                                      logLevels, maxRetries, minHeapRatio), writer);
+                                      getReservedMemory(), config.get(YarnConfiguration.RM_SCHEDULER_ADDRESS),
+                                      logLevels, maxRetries, getMinHeapRatio()), writer);
     }
     LOG.debug("Done {}", targetFile);
   }
@@ -787,7 +828,7 @@ final class YarnTwillPreparer implements TwillPreparer {
   }
 
   private ApplicationBundler createBundler(ClassAcceptor classAcceptor) {
-    return new ApplicationBundler(classAcceptor).setTempDir(localStagingDir);
+    return new ApplicationBundler(classAcceptor).setTempDir(getLocalStagingDir());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/twill/blob/e154bfed/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index 405eb24..d8e48de 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -126,7 +126,6 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
   };
 
   private final YarnConfiguration yarnConfig;
-  private final YarnAppClient yarnAppClient;
   private final ZKClientService zkClientService;
   private final LocationFactory locationFactory;
   private final Table<String, RunId, YarnTwillController> controllers;
@@ -162,7 +161,6 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
    */
   public YarnTwillRunnerService(YarnConfiguration config, String zkConnect, LocationFactory locationFactory) {
     this.yarnConfig = config;
-    this.yarnAppClient = new VersionDetectYarnAppClientFactory().create(config);
     this.locationFactory = locationFactory;
     this.zkClientService = getZKClientService(zkConnect);
     this.controllers = HashBasedTable.create();
@@ -288,8 +286,9 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
       locationCache = new NoCachingLocationCache(appLocation);
     }
 
-    return new YarnTwillPreparer(yarnConfig, twillSpec, runId, yarnAppClient,
-                                 zkClientService.getConnectString(), appLocation, twillClassPaths, jvmOptions,
+    Configuration config = new Configuration(yarnConfig);
+    return new YarnTwillPreparer(config, twillSpec, runId, zkClientService.getConnectString(),
+                                 appLocation, twillClassPaths, jvmOptions,
                                  locationCache, new YarnTwillControllerFactory() {
       @Override
       public YarnTwillController create(RunId runId, Iterable<LogHandler> logHandlers,
@@ -596,6 +595,8 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
         synchronized (YarnTwillRunnerService.this) {
           if (!controllers.contains(appName, runId)) {
             ZKClient zkClient = ZKClients.namespace(zkClientService, "/" + appName);
+            YarnAppClient yarnAppClient = new VersionDetectYarnAppClientFactory().create(new Configuration(yarnConfig));
+
             YarnTwillController controller = listenController(
               new YarnTwillController(appName, runId, zkClient, amLiveNodeData, yarnAppClient));
             controllers.put(appName, runId, controller);

http://git-wip-us.apache.org/repos/asf/twill/blob/e154bfed/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java
index c6f7b9a..f5143ce 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java
@@ -20,6 +20,8 @@ package org.apache.twill.yarn;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.Configs;
+import org.apache.twill.api.ResourceReport;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.TwillApplication;
 import org.apache.twill.api.TwillController;
@@ -33,6 +35,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.PrintWriter;
+import java.util.Collections;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -65,12 +68,19 @@ public class ContainerSizeTestRun extends BaseYarnTest {
   public void testMaxHeapSize() throws InterruptedException, TimeoutException, ExecutionException {
     TwillRunner runner = getTwillRunner();
     TwillController controller = runner.prepare(new MaxHeapApp())
+      // Alter the AM container size
+      .withConfiguration(Collections.singletonMap(Configs.Keys.YARN_AM_MEMORY_MB, "256"))
       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
       .start();
 
     try {
       ServiceDiscovered discovered = controller.discoverService("sleep");
       Assert.assertTrue(waitForSize(discovered, 1, 120));
+
+      // Verify the AM container size
+      ResourceReport resourceReport = controller.getResourceReport();
+      Assert.assertNotNull(resourceReport);
+      Assert.assertEquals(256, resourceReport.getAppMasterResources().getMemoryMB());
     } finally {
       controller.terminate().get(120, TimeUnit.SECONDS);
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/e154bfed/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java b/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
index 9daf06c..a141176 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
@@ -138,7 +138,7 @@ public class TwillTester extends ExternalResource {
                "org.apache.hadoop.yarn.util.resource.DominantResourceCalculator");
       conf.setBoolean("yarn.scheduler.include-port-in-node-name", true);
     }
-    conf.set("yarn.nodemanager.vmem-pmem-ratio", "20.1");
+    conf.set("yarn.nodemanager.vmem-pmem-ratio", "100.1");
     conf.set("yarn.nodemanager.vmem-check-enabled", "false");
     conf.set("yarn.scheduler.minimum-allocation-mb", "128");
     conf.set("yarn.nodemanager.delete.debug-delay-sec", "3600");


[15/24] twill git commit: (TWILL-189) Allows secure store update with different UGI

Posted by ch...@apache.org.
(TWILL-189) Allows secure store update with different UGI

- Deprecated the old TwillRunner.scheduleSecureStoreUpdate
- Added new TwillRunner.setSecureStoreRenewer method
  - Takes SecureStoreRenewer that writes SecureStore via
    SecureStoreWriter
    - The renewer implementation can use appropriate UGI.doAs to make
      call to SecureStoreWriter

This closes #48 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: e61be38fb071506de3d7172d1d0b903d559ada2c
Parents: 82e5942
Author: Terence Yim <ch...@apache.org>
Authored: Tue Mar 28 14:31:06 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Fri Mar 31 14:04:17 2017 -0700

----------------------------------------------------------------------
 .../apache/twill/api/SecureStoreUpdater.java    |   5 +
 .../java/org/apache/twill/api/TwillRunner.java  |  20 ++
 .../twill/api/security/SecureStoreRenewer.java  |  41 +++
 .../twill/api/security/SecureStoreWriter.java   |  36 +++
 .../twill/yarn/LocationSecureStoreRenewer.java  |  49 ++++
 .../twill/yarn/LocationSecureStoreUpdater.java  |  54 ----
 .../twill/yarn/YarnTwillRunnerService.java      | 278 ++++++++++++++-----
 7 files changed, 357 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/e61be38f/twill-api/src/main/java/org/apache/twill/api/SecureStoreUpdater.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/SecureStoreUpdater.java b/twill-api/src/main/java/org/apache/twill/api/SecureStoreUpdater.java
index 5912247..f468745 100644
--- a/twill-api/src/main/java/org/apache/twill/api/SecureStoreUpdater.java
+++ b/twill-api/src/main/java/org/apache/twill/api/SecureStoreUpdater.java
@@ -17,9 +17,14 @@
  */
 package org.apache.twill.api;
 
+import org.apache.twill.api.security.SecureStoreRenewer;
+
 /**
  * Represents class capable of creating update of {@link SecureStore} for live applications.
+ *
+ * @deprecated This class doesn't handle user correctly, use {@link SecureStoreRenewer} instead
  */
+@Deprecated
 public interface SecureStoreUpdater {
 
   /**

http://git-wip-us.apache.org/repos/asf/twill/blob/e61be38f/twill-api/src/main/java/org/apache/twill/api/TwillRunner.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillRunner.java b/twill-api/src/main/java/org/apache/twill/api/TwillRunner.java
index 845ced8..bdb812c 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillRunner.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillRunner.java
@@ -17,6 +17,8 @@
  */
 package org.apache.twill.api;
 
+import org.apache.twill.api.security.SecureStoreRenewer;
+import org.apache.twill.api.security.SecureStoreWriter;
 import org.apache.twill.common.Cancellable;
 
 import java.util.concurrent.TimeUnit;
@@ -105,7 +107,25 @@ public interface TwillRunner {
    * @param delay Delay between completion of one update call to the next one.
    * @param unit time unit for the initialDelay and delay.
    * @return A {@link Cancellable} for cancelling the scheduled update.
+   *
+   * @deprecated Use {@link #setSecureStoreRenewer(SecureStoreRenewer, long, long, long, TimeUnit)} instead.
    */
+  @Deprecated
   Cancellable scheduleSecureStoreUpdate(final SecureStoreUpdater updater,
                                         long initialDelay, long delay, TimeUnit unit);
+
+  /**
+   * Sets and schedules a periodic renewal of {@link SecureStore} using a given {@link SecureStoreRenewer}.
+   * There is always only one active {@link SecureStoreRenewer}. Setting a new renewer will replace the old one
+   * and setting up a new schedule.
+   *
+   * @param renewer a {@link SecureStoreRenewer} for renewing {@link SecureStore} for all applications.
+   * @param initialDelay delay before the first call to renew method.
+   * @param delay the delay between successful completion of one renew call to the next one.
+   * @param retryDelay the delay before the retrying the renewal if the call
+   *                   to {@link SecureStoreRenewer#renew(String, RunId, SecureStoreWriter)} raised exception.
+   * @param unit time unit for the initialDelay and period.
+   */
+  Cancellable setSecureStoreRenewer(SecureStoreRenewer renewer,
+                                    long initialDelay, long delay, long retryDelay, TimeUnit unit);
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/e61be38f/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreRenewer.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreRenewer.java b/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreRenewer.java
new file mode 100644
index 0000000..0cb9740
--- /dev/null
+++ b/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreRenewer.java
@@ -0,0 +1,41 @@
+/*
+ * 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.twill.api.security;
+
+import org.apache.twill.api.RunId;
+import org.apache.twill.api.SecureStore;
+import org.apache.twill.filesystem.Location;
+
+import java.io.IOException;
+
+/**
+ * This class is responsible for renewing the secure store used by application.
+ */
+public abstract class SecureStoreRenewer {
+
+  /**
+   * Renew the secure store for an application run. It must uses the {@link SecureStoreWriter} to update the
+   * {@link SecureStore}.
+   *
+   * @param application The name of the application.
+   * @param runId The runId of the live application.
+   * @param secureStoreWriter a {@link SecureStoreWriter} for writing out the new {@link SecureStore}.
+   * @throws IOException if renewal failed
+   */
+  public abstract void renew(String application, RunId runId, SecureStoreWriter secureStoreWriter) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/twill/blob/e61be38f/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreWriter.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreWriter.java b/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreWriter.java
new file mode 100644
index 0000000..fd8d04f
--- /dev/null
+++ b/twill-api/src/main/java/org/apache/twill/api/security/SecureStoreWriter.java
@@ -0,0 +1,36 @@
+/*
+ * 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.twill.api.security;
+
+import org.apache.twill.api.SecureStore;
+
+import java.io.IOException;
+
+/**
+ * A writer provided to {@link SecureStoreRenewer} for writing out {@link SecureStore} during renewal process.
+ */
+public interface SecureStoreWriter {
+
+  /**
+   * Writes the given {@link SecureStore}.
+   *
+   * @param secureStore the {@link SecureStore} to persist
+   * @throws IOException if failed to write out the {@link SecureStore}
+   */
+  void write(SecureStore secureStore) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/twill/blob/e61be38f/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreRenewer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreRenewer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreRenewer.java
new file mode 100644
index 0000000..5593777
--- /dev/null
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreRenewer.java
@@ -0,0 +1,49 @@
+/*
+ * 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.twill.yarn;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.twill.api.RunId;
+import org.apache.twill.api.security.SecureStoreRenewer;
+import org.apache.twill.api.security.SecureStoreWriter;
+import org.apache.twill.filesystem.LocationFactory;
+import org.apache.twill.internal.yarn.YarnUtils;
+
+import java.io.IOException;
+
+/**
+ * Package private class for updating location related secure store.
+ */
+final class LocationSecureStoreRenewer extends SecureStoreRenewer {
+
+  private final Configuration configuration;
+  private final LocationFactory locationFactory;
+
+  LocationSecureStoreRenewer(Configuration configuration, LocationFactory locationFactory) {
+    this.configuration = configuration;
+    this.locationFactory = locationFactory;
+  }
+
+  @Override
+  public void renew(String application, RunId runId, SecureStoreWriter secureStoreWriter) throws IOException {
+    Credentials credentials = new Credentials();
+    YarnUtils.addDelegationTokens(configuration, locationFactory, credentials);
+    secureStoreWriter.write(YarnSecureStore.create(credentials));
+  }
+}

http://git-wip-us.apache.org/repos/asf/twill/blob/e61be38f/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java b/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java
deleted file mode 100644
index 02fd356..0000000
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java
+++ /dev/null
@@ -1,54 +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.
- */
-package org.apache.twill.yarn;
-
-import com.google.common.base.Throwables;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.Credentials;
-import org.apache.twill.api.RunId;
-import org.apache.twill.api.SecureStore;
-import org.apache.twill.api.SecureStoreUpdater;
-import org.apache.twill.filesystem.LocationFactory;
-import org.apache.twill.internal.yarn.YarnUtils;
-
-import java.io.IOException;
-
-/**
- * Package private class for updating location related secure store.
- */
-final class LocationSecureStoreUpdater implements SecureStoreUpdater {
-
-  private final Configuration configuration;
-  private final LocationFactory locationFactory;
-
-  LocationSecureStoreUpdater(Configuration configuration, LocationFactory locationFactory) {
-    this.configuration = configuration;
-    this.locationFactory = locationFactory;
-  }
-
-  @Override
-  public SecureStore update(String application, RunId runId) {
-    try {
-      Credentials credentials = new Credentials();
-      YarnUtils.addDelegationTokens(configuration, locationFactory, credentials);
-      return YarnSecureStore.create(credentials);
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/twill/blob/e61be38f/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index a31265e..269ffdf 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -25,6 +25,7 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableTable;
 import com.google.common.collect.Iterables;
@@ -57,6 +58,8 @@ import org.apache.twill.api.TwillRunnable;
 import org.apache.twill.api.TwillRunnerService;
 import org.apache.twill.api.TwillSpecification;
 import org.apache.twill.api.logging.LogHandler;
+import org.apache.twill.api.security.SecureStoreRenewer;
+import org.apache.twill.api.security.SecureStoreWriter;
 import org.apache.twill.common.Cancellable;
 import org.apache.twill.common.Threads;
 import org.apache.twill.filesystem.FileContextLocationFactory;
@@ -91,6 +94,7 @@ import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URL;
 import java.util.HashSet;
@@ -99,6 +103,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
@@ -214,45 +219,29 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
   @Override
   public Cancellable scheduleSecureStoreUpdate(final SecureStoreUpdater updater,
                                                long initialDelay, long delay, TimeUnit unit) {
-    if (!UserGroupInformation.isSecurityEnabled()) {
-      return new Cancellable() {
-        @Override
-        public void cancel() {
-          // No-op
-        }
-      };
-    }
-
     synchronized (this) {
       if (secureStoreScheduler == null) {
         secureStoreScheduler = Executors.newSingleThreadScheduledExecutor(
-          Threads.createDaemonThreadFactory("secure-store-updater"));
+          Threads.createDaemonThreadFactory("secure-store-renewer"));
       }
     }
 
     final ScheduledFuture<?> future = secureStoreScheduler.scheduleWithFixedDelay(new Runnable() {
       @Override
       public void run() {
-        // Collects all <application, runId> pairs first
-        Multimap<String, RunId> liveApps = HashMultimap.create();
-        synchronized (YarnTwillRunnerService.this) {
-          for (Table.Cell<String, RunId, YarnTwillController> cell : controllers.cellSet()) {
-            liveApps.put(cell.getRowKey(), cell.getColumnKey());
-          }
+        // Collects all live applications
+        Table<String, RunId, YarnTwillController> liveApps;
+        synchronized (this) {
+          liveApps = HashBasedTable.create(controllers);
         }
 
-        // Collect all secure stores that needs to be updated.
-        Table<String, RunId, SecureStore> secureStores = HashBasedTable.create();
-        for (Map.Entry<String, RunId> entry : liveApps.entries()) {
-          try {
-            secureStores.put(entry.getKey(), entry.getValue(), updater.update(entry.getKey(), entry.getValue()));
-          } catch (Throwable t) {
-            LOG.warn("Exception thrown by SecureStoreUpdater {}", updater, t);
+        // Update the secure store with merging = true
+        renewSecureStore(liveApps, new SecureStoreRenewer() {
+          @Override
+          public void renew(String application, RunId runId, SecureStoreWriter secureStoreWriter) throws IOException {
+            secureStoreWriter.write(updater.update(application, runId));
           }
-        }
-
-        // Update secure stores.
-        updateSecureStores(secureStores);
+        }, true);
       }
     }, initialDelay, delay, unit);
 
@@ -265,6 +254,37 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
   }
 
   @Override
+  public Cancellable setSecureStoreRenewer(SecureStoreRenewer renewer, long initialDelay,
+                                           long delay, long retryDelay, TimeUnit unit) {
+    synchronized (this) {
+      if (secureStoreScheduler != null) {
+        // Shutdown and block until the schedule is stopped
+        stopScheduler(secureStoreScheduler);
+      }
+      secureStoreScheduler = Executors.newSingleThreadScheduledExecutor(
+        Threads.createDaemonThreadFactory("secure-store-renewer"));
+    }
+
+    final ScheduledExecutorService currentScheduler = secureStoreScheduler;
+    secureStoreScheduler.scheduleWithFixedDelay(
+      createSecureStoreUpdateRunnable(currentScheduler, renewer,
+                                      ImmutableMultimap.<String, RunId>of(), retryDelay, unit),
+      initialDelay, delay, unit);
+    return new Cancellable() {
+      @Override
+      public void cancel() {
+        synchronized (YarnTwillRunnerService.this) {
+          // Only cancel if the active scheduler is the same as the schedule bind to this cancellable
+          if (currentScheduler == secureStoreScheduler) {
+            secureStoreScheduler.shutdown();
+            secureStoreScheduler = null;
+          }
+        }
+      }
+    };
+  }
+
+  @Override
   public TwillPreparer prepare(TwillRunnable runnable) {
     return prepare(runnable, ResourceSpecification.BASIC);
   }
@@ -367,8 +387,9 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
       if (delay <= 0) {
         delay = (renewalInterval <= 2) ? 1 : renewalInterval / 2;
       }
-      scheduleSecureStoreUpdate(new LocationSecureStoreUpdater(yarnConfig, locationFactory),
-                                delay, delay, TimeUnit.MILLISECONDS);
+
+      setSecureStoreRenewer(new LocationSecureStoreRenewer(yarnConfig, locationFactory),
+                            delay, delay, 10000L, TimeUnit.MILLISECONDS);
     }
 
     // Optionally create a LocationCache
@@ -613,62 +634,109 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
     }, Threads.SAME_THREAD_EXECUTOR);
   }
 
-
-  private void updateSecureStores(Table<String, RunId, SecureStore> secureStores) {
-    for (Table.Cell<String, RunId, SecureStore> cell : secureStores.cellSet()) {
-      Object store = cell.getValue().getStore();
-      if (!(store instanceof Credentials)) {
-        LOG.warn("Only Hadoop Credentials is supported. Ignore update for {}.", cell);
-        continue;
+  /**
+   * Stops the given scheduler and block until is it stopped.
+   */
+  private void stopScheduler(final ScheduledExecutorService scheduler) {
+    scheduler.shutdown();
+    boolean interrupted = false;
+    try {
+      while (true) {
+        try {
+          scheduler.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
+          return;
+        } catch (InterruptedException e) {
+          interrupted = true;
+        }
       }
-
-      Credentials credentials = (Credentials) store;
-      if (credentials.getAllTokens().isEmpty()) {
-        // Nothing to update.
-        continue;
+    } finally {
+      if (interrupted) {
+        Thread.currentThread().interrupt();
       }
+    }
+  }
+
+  /**
+   * Creates a {@link Runnable} for renewing {@link SecureStore} for running applications.
+   *
+   * @param scheduler the schedule to schedule next renewal execution
+   * @param renewer the {@link SecureStoreRenewer} to use for renewal
+   * @param retryRuns if non-empty, only the given set of application name and run id that need to have
+   *                  secure store renewed; if empty, renew all running applications
+   * @param retryDelay the delay before retrying applications that are failed to have secure store renewed
+   * @param timeUnit the unit for the {@code delay} and {@code failureDelay}.
+   * @return a {@link Runnable}
+   */
+  private Runnable createSecureStoreUpdateRunnable(final ScheduledExecutorService scheduler,
+                                                   final SecureStoreRenewer renewer,
+                                                   final Multimap<String, RunId> retryRuns,
+                                                   final long retryDelay, final TimeUnit timeUnit) {
+    return new Runnable() {
+      @Override
+      public void run() {
+        // Collects the set of running application runs
+        Table<String, RunId, YarnTwillController> liveApps;
 
-      try {
-        updateCredentials(cell.getRowKey(), cell.getColumnKey(), credentials);
         synchronized (YarnTwillRunnerService.this) {
-          // Notify the application for secure store updates if it is still running.
-          YarnTwillController controller = controllers.get(cell.getRowKey(), cell.getColumnKey());
-          if (controller != null) {
-            controller.secureStoreUpdated();
+          if (retryRuns.isEmpty()) {
+            liveApps = HashBasedTable.create(controllers);
+          } else {
+            // If this is a renew retry, only renew the one in the retryRuns set
+            liveApps = HashBasedTable.create();
+            for (Table.Cell<String, RunId, YarnTwillController> cell : controllers.cellSet()) {
+              if (retryRuns.containsEntry(cell.getRowKey(), cell.getColumnKey())) {
+                liveApps.put(cell.getRowKey(), cell.getColumnKey(), cell.getValue());
+              }
+            }
           }
         }
-      } catch (Throwable t) {
-        LOG.warn("Failed to update secure store for {}.", cell, t);
-      }
-    }
-  }
 
-  private void updateCredentials(String application, RunId runId, Credentials updates) throws IOException {
-    Location credentialsLocation = locationFactory.create(String.format("/%s/%s/%s", application, runId.getId(),
-                                                                        Constants.Files.CREDENTIALS));
-    // Try to read the old credentials.
-    Credentials credentials = new Credentials();
-    if (credentialsLocation.exists()) {
-      try (DataInputStream is = new DataInputStream(new BufferedInputStream(credentialsLocation.getInputStream()))) {
-        credentials.readTokenStorageStream(is);
+        Multimap<String, RunId> failureRenews = renewSecureStore(liveApps, renewer, false);
+
+        if (!failureRenews.isEmpty()) {
+          // If there are failure during the renewal, schedule a retry with a new Runnable.
+          LOG.info("Schedule to retry on secure store renewal for applications {} in {} {}",
+                   failureRenews.keySet(), retryDelay, timeUnit.name().toLowerCase());
+          try {
+            scheduler.schedule(
+              createSecureStoreUpdateRunnable(scheduler, renewer, failureRenews, retryDelay, timeUnit),
+              retryDelay, timeUnit);
+          } catch (RejectedExecutionException e) {
+            // If the renewal is stopped, the scheduler will be stopped,
+            // hence this exception will be thrown and can be safely ignore.
+          }
+        }
       }
-    }
+    };
+  }
 
-    // Overwrite with the updates.
-    credentials.addAll(updates);
+  /**
+   * Renews the {@link SecureStore} for all the running applications.
+   *
+   * @param liveApps set of running applications that need to have secure store renewal
+   * @param renewer the {@link SecureStoreRenewer} for renewal
+   * @param mergeCredentials {@code true} to merge with existing credentials
+   * @return a {@link Multimap} containing the application runs that were failed to have secure store renewed
+   */
+  private Multimap<String, RunId> renewSecureStore(Table<String, RunId, YarnTwillController> liveApps,
+                                                   SecureStoreRenewer renewer, boolean mergeCredentials) {
+    Multimap<String, RunId> failureRenews = HashMultimap.create();
 
-    // Overwrite the credentials.
-    Location tmpLocation = credentialsLocation.getTempFile(Constants.Files.CREDENTIALS);
+    // Renew the secure store for each running application
+    for (Table.Cell<String, RunId, YarnTwillController> liveApp : liveApps.cellSet()) {
+      String application = liveApp.getRowKey();
+      RunId runId = liveApp.getColumnKey();
+      YarnTwillController controller = liveApp.getValue();
 
-    // Save the credentials store with user-only permission.
-    try (DataOutputStream os = new DataOutputStream(new BufferedOutputStream(tmpLocation.getOutputStream("600")))) {
-      credentials.writeTokenStorageToStream(os);
+      try {
+        renewer.renew(application, runId, new YarnSecureStoreWriter(application, runId, controller, mergeCredentials));
+      } catch (Exception e) {
+        LOG.warn("Failed to renew secure store for {}:{}", application, runId, e);
+        failureRenews.put(application, runId);
+      }
     }
 
-    // Rename the tmp file into the credentials location
-    tmpLocation.renameTo(credentialsLocation);
-
-    LOG.debug("Secure store for {} {} saved to {}.", application, runId, credentialsLocation);
+    return failureRenews;
   }
 
   private static LocationFactory createDefaultLocationFactory(Configuration configuration) {
@@ -680,4 +748,70 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
       throw Throwables.propagate(e);
     }
   }
+
+  /**
+   * A {@link SecureStoreWriter} for updating secure store for YARN application via a shared location with the
+   * running application.
+   */
+  private final class YarnSecureStoreWriter implements SecureStoreWriter {
+
+    private final String application;
+    private final RunId runId;
+    private final YarnTwillController controller;
+    private final boolean mergeCredentials;
+
+    private YarnSecureStoreWriter(String application, RunId runId,
+                                  YarnTwillController controller, boolean mergeCredentials) {
+      this.application = application;
+      this.runId = runId;
+      this.controller = controller;
+      this.mergeCredentials = mergeCredentials;
+    }
+
+    @Override
+    public void write(SecureStore secureStore) throws IOException {
+      Object store = secureStore.getStore();
+      if (!(store instanceof Credentials)) {
+        LOG.warn("Only Hadoop Credentials is supported. Ignore update for {}:{} with secure store {}",
+                 application, runId, secureStore);
+        return;
+      }
+
+      Location credentialsLocation = locationFactory.create(String.format("/%s/%s/%s", application, runId.getId(),
+                                                                          Constants.Files.CREDENTIALS));
+
+      LOG.debug("Writing new secure store for {}:{} to {}", application, runId, credentialsLocation);
+
+      Credentials credentials = new Credentials();
+      if (mergeCredentials) {
+        // Try to read the old credentials.
+        try (DataInputStream is = new DataInputStream(new BufferedInputStream(credentialsLocation.getInputStream()))) {
+          credentials.readTokenStorageStream(is);
+        } catch (FileNotFoundException e) {
+          // This is safe to ignore as the file may not be there
+        } catch (Exception e) {
+          // Just log and proceed.
+          LOG.warn("Failed to read existing credentials from {} for merging due to {}.",
+                   credentialsLocation, e.toString());
+        }
+      }
+
+      // Overwrite with credentials from the secure store
+      credentials.addAll((Credentials) store);
+      Location tmpLocation = credentialsLocation.getTempFile(Constants.Files.CREDENTIALS);
+
+      // Save the credentials store with user-only permission.
+      try (DataOutputStream os = new DataOutputStream(new BufferedOutputStream(tmpLocation.getOutputStream("600")))) {
+        credentials.writeTokenStorageToStream(os);
+      }
+
+      // Rename the tmp file into the credentials location
+      tmpLocation.renameTo(credentialsLocation);
+
+      // Notify the application that the credentials has been updated
+      controller.secureStoreUpdated();
+
+      LOG.debug("Secure store for {} {} saved to {}.", application, runId, credentialsLocation);
+    }
+  }
 }


[16/24] twill git commit: (TWILL-228) Remove the optimization when building application jar

Posted by ch...@apache.org.
(TWILL-228) Remove the optimization when building application jar

- The optimization can leading skipping certain jars in the application jar

This closes #50 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/2a316a60
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/2a316a60
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/2a316a60

Branch: refs/heads/site
Commit: 2a316a60b041f39bff95bfcd45f308f7c89d4bfa
Parents: e61be38
Author: Terence Yim <ch...@apache.org>
Authored: Fri Mar 31 15:39:20 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Fri Mar 31 16:15:14 2017 -0700

----------------------------------------------------------------------
 .../apache/twill/yarn/YarnTwillPreparer.java    | 23 +++-----------------
 .../twill/yarn/YarnTwillRunnerService.java      | 21 +-----------------
 2 files changed, 4 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/2a316a60/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index 4846fe3..c8abf4f 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -146,7 +146,6 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final Credentials credentials;
   private final Map<String, Map<String, String>> logLevels = Maps.newHashMap();
   private final LocationCache locationCache;
-  private final Set<URL> twillClassPaths;
   private String schedulerQueue;
   private String extraOptions;
   private JvmOptions.DebugOptions debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
@@ -154,8 +153,8 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final Map<String, Integer> maxRetries = Maps.newHashMap();
 
   YarnTwillPreparer(Configuration config, TwillSpecification twillSpec, RunId runId,
-                    String zkConnectString, Location appLocation, Set<URL> twillClassPaths,
-                    String extraOptions, LocationCache locationCache, YarnTwillControllerFactory controllerFactory) {
+                    String zkConnectString, Location appLocation, String extraOptions,
+                    LocationCache locationCache, YarnTwillControllerFactory controllerFactory) {
     this.config = config;
     this.twillSpec = twillSpec;
     this.runId = runId;
@@ -166,7 +165,6 @@ final class YarnTwillPreparer implements TwillPreparer {
     this.extraOptions = extraOptions;
     this.classAcceptor = new ClassAcceptor();
     this.locationCache = locationCache;
-    this.twillClassPaths = twillClassPaths;
   }
 
   private void confirmRunnableName(String runnableName) {
@@ -372,7 +370,7 @@ final class YarnTwillPreparer implements TwillPreparer {
 
             createLauncherJar(localFiles);
             createTwillJar(createBundler(classAcceptor), yarnAppClient, localFiles);
-            createApplicationJar(createApplicationJarBundler(classAcceptor), localFiles);
+            createApplicationJar(createBundler(classAcceptor), localFiles);
             createResourcesJar(createBundler(classAcceptor), localFiles);
 
             Path runtimeConfigDir = Files.createTempDirectory(getLocalStagingDir().toPath(),
@@ -835,19 +833,4 @@ final class YarnTwillPreparer implements TwillPreparer {
   private ApplicationBundler createBundler(ClassAcceptor classAcceptor) {
     return new ApplicationBundler(classAcceptor).setTempDir(getLocalStagingDir());
   }
-
-  /**
-   * Creates a {@link ApplicationBundler} for building application jar. The bundler will include classes
-   * accepted by the given {@link ClassAcceptor}, as long as it is not a twill class.
-   */
-  private ApplicationBundler createApplicationJarBundler(final ClassAcceptor classAcceptor) {
-    // Accept classes based on the classAcceptor and also excluding all twill classes as they are already
-    // in the twill.jar
-    return createBundler(new ClassAcceptor() {
-      @Override
-      public boolean accept(String className, URL classUrl, URL classPathUrl) {
-        return !twillClassPaths.contains(classPathUrl) && classAcceptor.accept(className, classUrl, classPathUrl);
-      }
-    });
-  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/2a316a60/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index 269ffdf..20627e2 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.twill.api.ClassAcceptor;
 import org.apache.twill.api.Configs;
 import org.apache.twill.api.LocalFile;
 import org.apache.twill.api.ResourceSpecification;
@@ -73,7 +72,6 @@ import org.apache.twill.internal.appmaster.ApplicationMasterLiveNodeData;
 import org.apache.twill.internal.io.BasicLocationCache;
 import org.apache.twill.internal.io.LocationCache;
 import org.apache.twill.internal.io.NoCachingLocationCache;
-import org.apache.twill.internal.utils.Dependencies;
 import org.apache.twill.internal.yarn.VersionDetectYarnAppClientFactory;
 import org.apache.twill.internal.yarn.YarnAppClient;
 import org.apache.twill.internal.yarn.YarnApplicationReport;
@@ -96,7 +94,6 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.URL;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
@@ -134,7 +131,6 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
   private final ZKClientService zkClientService;
   private final LocationFactory locationFactory;
   private final Table<String, RunId, YarnTwillController> controllers;
-  private final Set<URL> twillClassPaths;
   // A Guava service to help the state transition.
   private final Service serviceDelegate;
   private LocationCache locationCache;
@@ -169,7 +165,6 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
     this.locationFactory = locationFactory;
     this.zkClientService = getZKClientService(zkConnect);
     this.controllers = HashBasedTable.create();
-    this.twillClassPaths = new HashSet<>();
     this.serviceDelegate = new AbstractIdleService() {
       @Override
       protected void startUp() throws Exception {
@@ -308,8 +303,7 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
 
     Configuration config = new Configuration(yarnConfig);
     return new YarnTwillPreparer(config, twillSpec, runId, zkClientService.getConnectString(),
-                                 appLocation, twillClassPaths, jvmOptions,
-                                 locationCache, new YarnTwillControllerFactory() {
+                                 appLocation, jvmOptions, locationCache, new YarnTwillControllerFactory() {
       @Override
       public YarnTwillController create(RunId runId, boolean logCollectionEnabled, Iterable<LogHandler> logHandlers,
                                         Callable<ProcessController<YarnApplicationReport>> startUp,
@@ -355,19 +349,6 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
   private void startUp() throws Exception {
     zkClientService.startAndWait();
 
-    // Find all the classpaths for Twill classes. It is used for class filtering when building application jar
-    // in the YarnTwillPreparer
-    Dependencies.findClassDependencies(getClass().getClassLoader(), new ClassAcceptor() {
-      @Override
-      public boolean accept(String className, URL classUrl, URL classPathUrl) {
-        if (!className.startsWith("org.apache.twill.")) {
-          return false;
-        }
-        twillClassPaths.add(classPathUrl);
-        return true;
-      }
-    }, getClass().getName());
-
     // Create the root node, so that the namespace root would get created if it is missing
     // If the exception is caused by node exists, then it's ok. Otherwise propagate the exception.
     ZKOperations.ignoreError(zkClientService.create("/", null, CreateMode.PERSISTENT),


[02/24] twill git commit: (TWILL-207) Only use list of class names as the cache name

Posted by ch...@apache.org.
(TWILL-207) Only use list of class names as the cache name

- Also some indentation changes.

This closes #35 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/0cc3159f
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/0cc3159f
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/0cc3159f

Branch: refs/heads/site
Commit: 0cc3159fe529b93cb33f8cca3d2ba37b0012a455
Parents: f4df32d
Author: Terence Yim <ch...@apache.org>
Authored: Tue Feb 28 18:03:45 2017 -0800
Committer: Terence Yim <ch...@apache.org>
Committed: Thu Mar 2 22:56:41 2017 -0800

----------------------------------------------------------------------
 .../apache/twill/yarn/YarnTwillPreparer.java    | 123 ++++++++++---------
 1 file changed, 63 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/0cc3159f/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index 07594e3..6fbbf93 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -186,7 +186,7 @@ final class YarnTwillPreparer implements TwillPreparer {
   private void confirmRunnableName(String runnableName) {
     Preconditions.checkNotNull(runnableName);
     Preconditions.checkArgument(twillSpec.getRunnables().containsKey(runnableName),
-      "Runnable %s is not defined in the application.", runnableName);
+                                "Runnable %s is not defined in the application.", runnableName);
   }
 
   @Override
@@ -369,57 +369,59 @@ final class YarnTwillPreparer implements TwillPreparer {
       final ApplicationMasterInfo appMasterInfo = launcher.getContainerInfo();
       Callable<ProcessController<YarnApplicationReport>> submitTask =
         new Callable<ProcessController<YarnApplicationReport>>() {
-        @Override
-        public ProcessController<YarnApplicationReport> call() throws Exception {
-
-          // Local files needed by AM
-          Map<String, LocalFile> localFiles = Maps.newHashMap();
-
-          createLauncherJar(localFiles);
-          createTwillJar(createBundler(classAcceptor), localFiles);
-          createApplicationJar(createApplicationJarBundler(classAcceptor), localFiles);
-          createResourcesJar(createBundler(classAcceptor), localFiles);
-
-          Path runtimeConfigDir = Files.createTempDirectory(localStagingDir.toPath(),
-                                                            Constants.Files.RUNTIME_CONFIG_JAR);
-          try {
-            saveSpecification(twillSpec, runtimeConfigDir.resolve(Constants.Files.TWILL_SPEC));
-            saveLogback(runtimeConfigDir.resolve(Constants.Files.LOGBACK_TEMPLATE));
-            saveClassPaths(runtimeConfigDir);
-            saveJvmOptions(runtimeConfigDir.resolve(Constants.Files.JVM_OPTIONS));
-            saveArguments(new Arguments(arguments, runnableArgs), runtimeConfigDir.resolve(Constants.Files.ARGUMENTS));
-            saveEnvironments(runtimeConfigDir.resolve(Constants.Files.ENVIRONMENTS));
-            createRuntimeConfigJar(runtimeConfigDir, localFiles);
-          } finally {
-            Paths.deleteRecursively(runtimeConfigDir);
-          }
+          @Override
+          public ProcessController<YarnApplicationReport> call() throws Exception {
+
+            // Local files needed by AM
+            Map<String, LocalFile> localFiles = Maps.newHashMap();
+
+            createLauncherJar(localFiles);
+            createTwillJar(createBundler(classAcceptor), localFiles);
+            createApplicationJar(createApplicationJarBundler(classAcceptor), localFiles);
+            createResourcesJar(createBundler(classAcceptor), localFiles);
+
+            Path runtimeConfigDir = Files.createTempDirectory(localStagingDir.toPath(),
+                                                              Constants.Files.RUNTIME_CONFIG_JAR);
+            try {
+              saveSpecification(twillSpec, runtimeConfigDir.resolve(Constants.Files.TWILL_SPEC));
+              saveLogback(runtimeConfigDir.resolve(Constants.Files.LOGBACK_TEMPLATE));
+              saveClassPaths(runtimeConfigDir);
+              saveJvmOptions(runtimeConfigDir.resolve(Constants.Files.JVM_OPTIONS));
+              saveArguments(new Arguments(arguments, runnableArgs),
+                            runtimeConfigDir.resolve(Constants.Files.ARGUMENTS));
+              saveEnvironments(runtimeConfigDir.resolve(Constants.Files.ENVIRONMENTS));
+              createRuntimeConfigJar(runtimeConfigDir, localFiles);
+            } finally {
+              Paths.deleteRecursively(runtimeConfigDir);
+            }
 
-          createLocalizeFilesJson(localFiles);
-
-          LOG.debug("Submit AM container spec: {}", appMasterInfo);
-          // java -Djava.io.tmpdir=tmp -cp launcher.jar:$HADOOP_CONF_DIR -XmxMemory
-          //     org.apache.twill.internal.TwillLauncher
-          //     appMaster.jar
-          //     org.apache.twill.internal.appmaster.ApplicationMasterMain
-          //     false
-
-          int memory = Resources.computeMaxHeapSize(appMasterInfo.getMemoryMB(),
-                                                    Constants.APP_MASTER_RESERVED_MEMORY_MB, Constants.HEAP_MIN_RATIO);
-          return launcher.prepareLaunch(ImmutableMap.<String, String>of(), localFiles.values(), credentials)
-            .addCommand(
-              "$JAVA_HOME/bin/java",
-              "-Djava.io.tmpdir=tmp",
-              "-Dyarn.appId=$" + EnvKeys.YARN_APP_ID_STR,
-              "-Dtwill.app=$" + Constants.TWILL_APP_NAME,
-              "-cp", Constants.Files.LAUNCHER_JAR + ":$HADOOP_CONF_DIR",
-              "-Xmx" + memory + "m",
-              extraOptions == null ? "" : extraOptions,
-              TwillLauncher.class.getName(),
-              ApplicationMasterMain.class.getName(),
-              Boolean.FALSE.toString())
-            .launch();
-        }
-      };
+            createLocalizeFilesJson(localFiles);
+
+            LOG.debug("Submit AM container spec: {}", appMasterInfo);
+            // java -Djava.io.tmpdir=tmp -cp launcher.jar:$HADOOP_CONF_DIR -XmxMemory
+            //     org.apache.twill.internal.TwillLauncher
+            //     appMaster.jar
+            //     org.apache.twill.internal.appmaster.ApplicationMasterMain
+            //     false
+
+            int memory = Resources.computeMaxHeapSize(appMasterInfo.getMemoryMB(),
+                                                      Constants.APP_MASTER_RESERVED_MEMORY_MB,
+                                                      Constants.HEAP_MIN_RATIO);
+            return launcher.prepareLaunch(ImmutableMap.<String, String>of(), localFiles.values(), credentials)
+              .addCommand(
+                "$JAVA_HOME/bin/java",
+                "-Djava.io.tmpdir=tmp",
+                "-Dyarn.appId=$" + EnvKeys.YARN_APP_ID_STR,
+                "-Dtwill.app=$" + Constants.TWILL_APP_NAME,
+                "-cp", Constants.Files.LAUNCHER_JAR + ":$HADOOP_CONF_DIR",
+                "-Xmx" + memory + "m",
+                extraOptions == null ? "" : extraOptions,
+                TwillLauncher.class.getName(),
+                ApplicationMasterMain.class.getName(),
+                Boolean.FALSE.toString())
+              .launch();
+          }
+        };
 
       YarnTwillController controller = controllerFactory.create(runId, logHandlers, submitTask, timeout, timeoutUnit);
       controller.start();
@@ -487,7 +489,7 @@ final class YarnTwillPreparer implements TwillPreparer {
       public void load(String name, Location targetLocation) throws IOException {
         // Stuck in the yarnAppClient class to make bundler being able to pickup the right yarn-client version
         bundler.createBundle(targetLocation, ApplicationMasterMain.class,
-          yarnAppClient.getClass(), TwillContainerMain.class, OptionSpec.class);
+                             yarnAppClient.getClass(), TwillContainerMain.class, OptionSpec.class);
       }
     });
 
@@ -519,7 +521,8 @@ final class YarnTwillPreparer implements TwillPreparer {
       for (String name : classList) {
         hasher.putString(name);
       }
-      String name = twillSpec.getName() + "-" + hasher.hash().toString() + "-" + Constants.Files.APPLICATION_JAR;
+      // Only depends on class list so that it can be reused across different launches
+      String name = hasher.hash().toString() + "-" + Constants.Files.APPLICATION_JAR;
 
       LOG.debug("Create and copy {}", Constants.Files.APPLICATION_JAR);
       Location location = locationCache.get(name, new LocationCache.Loader() {
@@ -612,12 +615,12 @@ final class YarnTwillPreparer implements TwillPreparer {
     // Rewrite LocalFiles inside twillSpec
     Map<String, RuntimeSpecification> runtimeSpec = Maps.transformEntries(
       spec.getRunnables(), new Maps.EntryTransformer<String, RuntimeSpecification, RuntimeSpecification>() {
-      @Override
-      public RuntimeSpecification transformEntry(String key, RuntimeSpecification value) {
-        return new DefaultRuntimeSpecification(value.getName(), value.getRunnableSpecification(),
-                                               value.getResourceSpecification(), runnableLocalFiles.get(key));
-      }
-    });
+        @Override
+        public RuntimeSpecification transformEntry(String key, RuntimeSpecification value) {
+          return new DefaultRuntimeSpecification(value.getName(), value.getRunnableSpecification(),
+                                                 value.getResourceSpecification(), runnableLocalFiles.get(key));
+        }
+      });
 
     // Serialize into a local temp file.
     LOG.debug("Creating {}", targetFile);
@@ -661,7 +664,7 @@ final class YarnTwillPreparer implements TwillPreparer {
       @Override
       public void load(String name, Location targetLocation) throws IOException {
         // Create a jar file with the TwillLauncher and FindFreePort and dependent classes inside.
-        try (final JarOutputStream jarOut = new JarOutputStream(targetLocation.getOutputStream())) {
+        try (JarOutputStream jarOut = new JarOutputStream(targetLocation.getOutputStream())) {
           ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
           if (classLoader == null) {
             classLoader = getClass().getClassLoader();


[17/24] twill git commit: (TWILL-179) Added support for custom ClassLoader for containers

Posted by ch...@apache.org.
(TWILL-179) Added support for custom ClassLoader for containers

- Added method TwillPreparer.setClassLoader
- Use system property "twill.custom.class.loader" to pass the class name of the custom ClassLoader

This closes #51 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: c8e2a615a2450c85e9344b50ee9ded562a54d018
Parents: 2a316a6
Author: Terence Yim <ch...@apache.org>
Authored: Mon Apr 3 12:34:14 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Apr 3 13:12:38 2017 -0700

----------------------------------------------------------------------
 .../org/apache/twill/api/TwillPreparer.java     | 13 +++
 .../org/apache/twill/internal/Constants.java    |  5 ++
 .../apache/twill/launcher/TwillLauncher.java    | 44 +++++++---
 .../apache/twill/yarn/YarnTwillPreparer.java    | 32 +++++--
 .../apache/twill/yarn/CustomClassLoader.java    | 87 ++++++++++++++++++++
 .../twill/yarn/CustomClassLoaderRunnable.java   | 56 +++++++++++++
 .../twill/yarn/CustomClassLoaderTestRun.java    | 42 ++++++++++
 .../org/apache/twill/yarn/YarnTestSuite.java    |  1 +
 8 files changed, 264 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java b/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
index 43b751b..1f50972 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
@@ -21,6 +21,7 @@ import org.apache.twill.api.logging.LogEntry;
 import org.apache.twill.api.logging.LogHandler;
 
 import java.net.URI;
+import java.net.URL;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
@@ -276,6 +277,18 @@ public interface TwillPreparer {
   TwillPreparer setLogLevels(String runnableName, Map<String, LogEntry.Level> logLevelsForRunnable);
 
   /**
+   * Sets the class name of the {@link ClassLoader} to be used for loading twill and application classes for
+   * all containers. The {@link ClassLoader} class should have a public constructor that takes two parameters in the
+   * form of {@code (URL[] urls, ClassLoader parentClassLoader)}.
+   * The first parameter is an array of {@link URL} that contains the list of {@link URL} for loading classes and
+   * resources; the second parameter is the parent {@link ClassLoader}.
+   *
+   * @param classLoaderClassName name of the {@link ClassLoader} class.
+   * @return This {@link TwillPreparer}.
+   */
+  TwillPreparer setClassLoader(String classLoaderClassName);
+
+  /**
    * Starts the application. It's the same as calling {@link #start(long, TimeUnit)} with timeout of 60 seconds.
    *
    * @return A {@link TwillController} for controlling the running application.

http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-common/src/main/java/org/apache/twill/internal/Constants.java
----------------------------------------------------------------------
diff --git a/twill-common/src/main/java/org/apache/twill/internal/Constants.java b/twill-common/src/main/java/org/apache/twill/internal/Constants.java
index 6e799d5..4135c9a 100644
--- a/twill-common/src/main/java/org/apache/twill/internal/Constants.java
+++ b/twill-common/src/main/java/org/apache/twill/internal/Constants.java
@@ -53,6 +53,11 @@ public final class Constants {
   public static final String TWILL_APP_NAME = "TWILL_APP_NAME";
 
   /**
+   * Constant for the system property name that carries the class name for the container ClassLoader as defined by user.
+   */
+  public static final String TWILL_CONTAINER_CLASSLOADER = "twill.container.class.loader";
+
+  /**
    * Constants for names of internal files that are shared between client, AM and containers.
    */
   public static final class Files {

http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-core/src/main/java/org/apache/twill/launcher/TwillLauncher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/launcher/TwillLauncher.java b/twill-core/src/main/java/org/apache/twill/launcher/TwillLauncher.java
index a5052b3..056639f 100644
--- a/twill-core/src/main/java/org/apache/twill/launcher/TwillLauncher.java
+++ b/twill-core/src/main/java/org/apache/twill/launcher/TwillLauncher.java
@@ -22,13 +22,11 @@ import org.apache.twill.internal.Constants;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URLClassLoader;
-import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.util.ArrayList;
@@ -61,11 +59,12 @@ public final class TwillLauncher {
     boolean userClassPath = Boolean.parseBoolean(args[1]);
 
     // Create ClassLoader
-    URLClassLoader classLoader = createClassLoader(userClassPath);
-    Thread.currentThread().setContextClassLoader(classLoader);
+    URL[] classpath = createClasspath(userClassPath);
+    ClassLoader classLoader = createContainerClassLoader(classpath);
+    System.out.println("Launch class (" + mainClassName + ") using classloader " + classLoader.getClass().getName()
+                         + " with classpath: " + Arrays.toString(classpath));
 
-    System.out.println("Launch class (" + mainClassName + ") with classpath: " +
-                         Arrays.toString(classLoader.getURLs()));
+    Thread.currentThread().setContextClassLoader(classLoader);
 
     Class<?> mainClass = classLoader.loadClass(mainClassName);
     Method mainMethod = mainClass.getMethod("main", String[].class);
@@ -77,7 +76,7 @@ public final class TwillLauncher {
     System.out.println("Launcher completed");
   }
 
-  private static URLClassLoader createClassLoader(boolean useClassPath) throws Exception {
+  private static URL[] createClasspath(boolean useClassPath) throws IOException {
     List<URL> urls = new ArrayList<>();
 
     File appJarDir = new File(Constants.Files.APPLICATION_JAR);
@@ -116,7 +115,33 @@ public final class TwillLauncher {
     }
 
     addClassPathsToList(urls, new File(Constants.Files.RUNTIME_CONFIG_JAR, Constants.Files.APPLICATION_CLASSPATH));
-    return new URLClassLoader(urls.toArray(new URL[urls.size()]));
+    return urls.toArray(new URL[urls.size()]);
+  }
+
+  /**
+   * Creates a {@link ClassLoader} to be used by this container that load classes from the given classpath.
+   */
+  private static ClassLoader createContainerClassLoader(URL[] classpath) {
+    String containerClassLoaderName = System.getProperty(Constants.TWILL_CONTAINER_CLASSLOADER);
+    URLClassLoader classLoader = new URLClassLoader(classpath);
+    if (containerClassLoaderName == null) {
+      return classLoader;
+    }
+
+    try {
+      @SuppressWarnings("unchecked")
+      Class<? extends ClassLoader> cls = (Class<? extends ClassLoader>) classLoader.loadClass(containerClassLoaderName);
+
+      // Instantiate with constructor (URL[] classpath, ClassLoader parentClassLoader)
+      return cls.getConstructor(URL[].class, ClassLoader.class).newInstance(classpath, classLoader.getParent());
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException("Failed to load container class loader class " + containerClassLoaderName, e);
+    } catch (NoSuchMethodException e) {
+      throw new RuntimeException("Container class loader must have a public constructor with " +
+                                   "parameters (URL[] classpath, ClassLoader parent)", e);
+    } catch (InstantiationException | InvocationTargetException | IllegalAccessException e) {
+      throw new RuntimeException("Failed to create container class loader of class " + containerClassLoaderName, e);
+    }
   }
 
   private static void addClassPathsToList(List<URL> urls, File classpathFile) throws IOException {
@@ -172,7 +197,6 @@ public final class TwillLauncher {
    * Populates a list of {@link File} under the given directory that has ".jar" as extension.
    */
   private static List<File> listJarFiles(File dir, List<File> result) {
-    System.out.println("listing jars for " + dir.getAbsolutePath());
     File[] files = dir.listFiles();
     if (files == null || files.length == 0) {
       return result;

http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index c8abf4f..52e18eb 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -146,11 +146,12 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final Credentials credentials;
   private final Map<String, Map<String, String>> logLevels = Maps.newHashMap();
   private final LocationCache locationCache;
+  private final Map<String, Integer> maxRetries = Maps.newHashMap();
   private String schedulerQueue;
   private String extraOptions;
   private JvmOptions.DebugOptions debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
   private ClassAcceptor classAcceptor;
-  private final Map<String, Integer> maxRetries = Maps.newHashMap();
+  private String classLoaderClassName;
 
   YarnTwillPreparer(Configuration config, TwillSpecification twillSpec, RunId runId,
                     String zkConnectString, Location appLocation, String extraOptions,
@@ -350,6 +351,12 @@ final class YarnTwillPreparer implements TwillPreparer {
   }
 
   @Override
+  public TwillPreparer setClassLoader(String classLoaderClassName) {
+    this.classLoaderClassName = classLoaderClassName;
+    return this;
+  }
+
+  @Override
   public TwillController start() {
     return start(Constants.APPLICATION_MAX_START_SECONDS, TimeUnit.SECONDS);
   }
@@ -365,6 +372,8 @@ final class YarnTwillPreparer implements TwillPreparer {
           @Override
           public ProcessController<YarnApplicationReport> call() throws Exception {
 
+            String extraOptions = getExtraOptions();
+
             // Local files needed by AM
             Map<String, LocalFile> localFiles = Maps.newHashMap();
 
@@ -379,7 +388,7 @@ final class YarnTwillPreparer implements TwillPreparer {
               saveSpecification(twillSpec, runtimeConfigDir.resolve(Constants.Files.TWILL_SPEC));
               saveLogback(runtimeConfigDir.resolve(Constants.Files.LOGBACK_TEMPLATE));
               saveClassPaths(runtimeConfigDir);
-              saveJvmOptions(runtimeConfigDir.resolve(Constants.Files.JVM_OPTIONS));
+              saveJvmOptions(extraOptions, debugOptions, runtimeConfigDir.resolve(Constants.Files.JVM_OPTIONS));
               saveArguments(new Arguments(arguments, runnableArgs),
                             runtimeConfigDir.resolve(Constants.Files.ARGUMENTS));
               saveEnvironments(runtimeConfigDir.resolve(Constants.Files.ENVIRONMENTS));
@@ -409,7 +418,7 @@ final class YarnTwillPreparer implements TwillPreparer {
                 "-Dtwill.app=$" + Constants.TWILL_APP_NAME,
                 "-cp", Constants.Files.LAUNCHER_JAR + ":$HADOOP_CONF_DIR",
                 "-Xmx" + memory + "m",
-                extraOptions == null ? "" : extraOptions,
+                extraOptions,
                 TwillLauncher.class.getName(),
                 ApplicationMasterMain.class.getName(),
                 Boolean.FALSE.toString())
@@ -452,6 +461,17 @@ final class YarnTwillPreparer implements TwillPreparer {
     return new File(config.get(Configs.Keys.LOCAL_STAGING_DIRECTORY, Configs.Defaults.LOCAL_STAGING_DIRECTORY));
   }
 
+  /**
+   * Returns the extra options for the container JVM.
+   */
+  private String getExtraOptions() {
+    String extraOptions = this.extraOptions == null ? "" : this.extraOptions;
+    if (classLoaderClassName != null) {
+      extraOptions += " -D" + Constants.TWILL_CONTAINER_CLASSLOADER + "=" + classLoaderClassName;
+    }
+    return extraOptions;
+  }
+
   private void setEnv(String runnableName, Map<String, String> env, boolean overwrite) {
     Map<String, String> environment = environments.get(runnableName);
     if (environment == null) {
@@ -742,9 +762,9 @@ final class YarnTwillPreparer implements TwillPreparer {
                 Joiner.on(':').join(classPaths).getBytes(StandardCharsets.UTF_8));
   }
 
-  private void saveJvmOptions(final Path targetPath) throws IOException {
-    if ((extraOptions == null || extraOptions.isEmpty()) &&
-      JvmOptions.DebugOptions.NO_DEBUG.equals(debugOptions)) {
+  private void saveJvmOptions(String extraOptions,
+                              JvmOptions.DebugOptions debugOptions, final Path targetPath) throws IOException {
+    if (extraOptions.isEmpty() && JvmOptions.DebugOptions.NO_DEBUG.equals(debugOptions)) {
       // If no vm options, no need to localize the file.
       return;
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoader.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoader.java b/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoader.java
new file mode 100644
index 0000000..51c1dc0
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoader.java
@@ -0,0 +1,87 @@
+/*
+ * 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.twill.yarn;
+
+import org.apache.twill.api.ServiceAnnouncer;
+import org.apache.twill.common.Cancellable;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.Type;
+import org.objectweb.asm.commons.GeneratorAdapter;
+import org.objectweb.asm.commons.Method;
+
+import java.net.URL;
+import java.net.URLClassLoader;
+
+/**
+ * ClassLoader that generates a new class for the {@link CustomClassLoaderTestRun}.
+ */
+public final class CustomClassLoader extends URLClassLoader {
+
+  public CustomClassLoader(URL[] urls, ClassLoader parent) {
+    super(urls, parent);
+  }
+
+  @Override
+  protected Class<?> findClass(String name) throws ClassNotFoundException {
+    if (!CustomClassLoaderRunnable.GENERATED_CLASS_NAME.equals(name)) {
+      return super.findClass(name);
+    }
+
+    // Generate a class that look like this:
+    //
+    // public class Generated {
+    //
+    //   public void announce(ServiceAnnouncer announcer, String serviceName, int port) {
+    //     announcer.announce(serviceName, port);
+    //   }
+    // }
+    Type generatedClassType = Type.getObjectType(CustomClassLoaderRunnable.GENERATED_CLASS_NAME.replace('.', '/'));
+    ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
+    cw.visit(Opcodes.V1_7, Opcodes.ACC_PUBLIC + Opcodes.ACC_FINAL,
+             generatedClassType.getInternalName(), null, Type.getInternalName(Object.class), null);
+
+    // Generate the default constructor, which just call super();
+    Method constructor = new Method("<init>", Type.VOID_TYPE, new Type[0]);
+    GeneratorAdapter mg = new GeneratorAdapter(Opcodes.ACC_PUBLIC, constructor, null, null, cw);
+    mg.loadThis();
+    mg.invokeConstructor(Type.getType(Object.class), constructor);
+    mg.returnValue();
+    mg.endMethod();
+
+    // Generate the announce method
+    Method announce = new Method("announce", Type.VOID_TYPE, new Type[] {
+      Type.getType(ServiceAnnouncer.class), Type.getType(String.class), Type.INT_TYPE
+    });
+    mg = new GeneratorAdapter(Opcodes.ACC_PUBLIC, announce, null, null, cw);
+    mg.loadArg(0);
+    mg.loadArg(1);
+    mg.loadArg(2);
+    mg.invokeInterface(Type.getType(ServiceAnnouncer.class),
+                       new Method("announce", Type.getType(Cancellable.class), new Type[] {
+                         Type.getType(String.class), Type.INT_TYPE
+                       }));
+    mg.pop();
+    mg.returnValue();
+    mg.endMethod();
+    cw.visitEnd();
+
+    byte[] byteCode = cw.toByteArray();
+    return defineClass(CustomClassLoaderRunnable.GENERATED_CLASS_NAME, byteCode, 0, byteCode.length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderRunnable.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderRunnable.java b/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderRunnable.java
new file mode 100644
index 0000000..66bcd42
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderRunnable.java
@@ -0,0 +1,56 @@
+/*
+ * 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.twill.yarn;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.ServiceAnnouncer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CountDownLatch;
+
+/**
+ * Runnable for testing custom classloader
+ */
+public final class CustomClassLoaderRunnable extends AbstractTwillRunnable {
+
+  static final String SERVICE_NAME = "custom.service";
+  static final String GENERATED_CLASS_NAME = "org.apache.twill.test.Generated";
+
+  private static final Logger LOG = LoggerFactory.getLogger(CustomClassLoaderRunnable.class);
+
+  private final CountDownLatch stopLatch = new CountDownLatch(1);
+
+  @Override
+  public void run() {
+    try {
+      Class<?> cls = Class.forName(GENERATED_CLASS_NAME);
+      java.lang.reflect.Method announce = cls.getMethod("announce", ServiceAnnouncer.class, String.class, int.class);
+      announce.invoke(cls.newInstance(), getContext(), SERVICE_NAME, 54321);
+      Uninterruptibles.awaitUninterruptibly(stopLatch);
+    } catch (Exception e) {
+      LOG.error("Failed to call announce on " + GENERATED_CLASS_NAME, e);
+    }
+  }
+
+  @Override
+  public void stop() {
+    stopLatch.countDown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderTestRun.java
new file mode 100644
index 0000000..0ac43a6
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/CustomClassLoaderTestRun.java
@@ -0,0 +1,42 @@
+/*
+ * 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.twill.yarn;
+
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+
+/**
+ * Unit test for testing custom classloader for containers.
+ */
+public class CustomClassLoaderTestRun extends BaseYarnTest {
+
+  @Test
+  public void testCustomClassLoader() throws Exception {
+    TwillController controller = getTwillRunner().prepare(new CustomClassLoaderRunnable())
+      .setClassLoader(CustomClassLoader.class.getName())
+      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
+      .start();
+
+    Assert.assertTrue(waitForSize(controller.discoverService(CustomClassLoaderRunnable.SERVICE_NAME), 1, 120));
+    controller.terminate().get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/twill/blob/c8e2a615/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
index 56172da..0911a3d 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
@@ -26,6 +26,7 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
   ContainerSizeTestRun.class,
+  CustomClassLoaderTestRun.class,
   DebugTestRun.class,
   DistributeShellTestRun.class,
   EchoServerTestRun.class,


[23/24] twill git commit: Update copyright year.

Posted by ch...@apache.org.
Update copyright year.


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/401496a7
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/401496a7
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/401496a7

Branch: refs/heads/site
Commit: 401496a7e6996335c5190fc7563a87d0d9008c7e
Parents: 73139ad
Author: Terence Yim <ch...@apache.org>
Authored: Fri Apr 7 11:40:36 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Fri Apr 7 11:40:41 2017 -0700

----------------------------------------------------------------------
 NOTICE | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/401496a7/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index a88b8d4..c899008 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Twill
-Copyright 2013-2016 The Apache Software Foundation
+Copyright 2013-2017 The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).


[21/24] twill git commit: Prepare for releasing 0.11.0

Posted by ch...@apache.org.
Prepare for releasing 0.11.0


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

Branch: refs/heads/site
Commit: e8fd829cbb0477d3e8f6c44fc554805c9dbec4f7
Parents: cc79f0d
Author: Terence Yim <ch...@apache.org>
Authored: Tue Apr 4 02:34:20 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Tue Apr 4 02:34:20 2017 -0700

----------------------------------------------------------------------
 pom.xml                      | 2 +-
 twill-api/pom.xml            | 2 +-
 twill-common/pom.xml         | 2 +-
 twill-core/pom.xml           | 2 +-
 twill-discovery-api/pom.xml  | 2 +-
 twill-discovery-core/pom.xml | 2 +-
 twill-examples/echo/pom.xml  | 2 +-
 twill-examples/pom.xml       | 2 +-
 twill-examples/yarn/pom.xml  | 2 +-
 twill-ext/pom.xml            | 2 +-
 twill-java8-test/pom.xml     | 2 +-
 twill-yarn/pom.xml           | 2 +-
 twill-zookeeper/pom.xml      | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b641b49..680d9a7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.twill</groupId>
     <artifactId>twill-parent</artifactId>
-    <version>0.11.0-SNAPSHOT</version>
+    <version>0.11.0</version>
     <packaging>pom</packaging>
     <name>Apache Twill</name>
     <url>http://twill.apache.org</url>

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index b1382d2..14afaee 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
 
     <artifactId>twill-api</artifactId>

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index 75f2d50..000871c 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index f8dcc14..90dc20f 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index a4adf57..eae740f 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index e561e63..6d5ee06 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-examples/echo/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/echo/pom.xml b/twill-examples/echo/pom.xml
index fc8515e..63ac83b 100644
--- a/twill-examples/echo/pom.xml
+++ b/twill-examples/echo/pom.xml
@@ -25,7 +25,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-examples</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
 
     <name>Apache Twill examples: Echo</name>

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-examples/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/pom.xml b/twill-examples/pom.xml
index 94bcd2a..84e65c6 100644
--- a/twill-examples/pom.xml
+++ b/twill-examples/pom.xml
@@ -25,7 +25,7 @@ limitations under the License.
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
 
     <artifactId>twill-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-examples/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/yarn/pom.xml b/twill-examples/yarn/pom.xml
index 16f107d..2a8243c 100644
--- a/twill-examples/yarn/pom.xml
+++ b/twill-examples/yarn/pom.xml
@@ -24,7 +24,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-examples</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
 
     <name>Apache Twill examples: YARN</name>

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-ext/pom.xml
----------------------------------------------------------------------
diff --git a/twill-ext/pom.xml b/twill-ext/pom.xml
index ef8d3b3..18abe8a 100644
--- a/twill-ext/pom.xml
+++ b/twill-ext/pom.xml
@@ -22,7 +22,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-java8-test/pom.xml
----------------------------------------------------------------------
diff --git a/twill-java8-test/pom.xml b/twill-java8-test/pom.xml
index a04e87b..7f21aad 100644
--- a/twill-java8-test/pom.xml
+++ b/twill-java8-test/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
 
     <artifactId>twill-java8-test</artifactId>

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index b2fa4c5..786d295 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/e8fd829c/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index 2961585..ed6df67 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0-SNAPSHOT</version>
+        <version>0.11.0</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[13/24] twill git commit: (TWILL-223) Make FileContextLocationFactory UGI aware

Posted by ch...@apache.org.
(TWILL-223) Make FileContextLocationFactory UGI aware

- Use different FileContext object based on the caller UGI
- Allows sharing the same factory instance for different user

This closes #47 on GitHub.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/2c42097c
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/2c42097c
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/2c42097c

Branch: refs/heads/site
Commit: 2c42097c5e0de2ef901b96458e0f2745c1d7aeb5
Parents: 4e1cae3
Author: Terence Yim <ch...@apache.org>
Authored: Tue Mar 28 12:53:43 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Thu Mar 30 01:16:28 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/twill/api/Configs.java |  10 +
 .../filesystem/FileContextLocationFactory.java  |  77 +++-
 .../twill/yarn/YarnTwillRunnerService.java      |   3 +-
 .../filesystem/FileContextLocationTest.java     |  58 +--
 .../twill/filesystem/LocalLocationTest.java     |  23 +-
 .../twill/filesystem/LocationTestBase.java      | 367 ++++++++++---------
 .../java/org/apache/twill/yarn/TwillTester.java |   3 +-
 7 files changed, 292 insertions(+), 249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/2c42097c/twill-api/src/main/java/org/apache/twill/api/Configs.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/Configs.java b/twill-api/src/main/java/org/apache/twill/api/Configs.java
index 1447a37..9a21489 100644
--- a/twill-api/src/main/java/org/apache/twill/api/Configs.java
+++ b/twill-api/src/main/java/org/apache/twill/api/Configs.java
@@ -83,6 +83,11 @@ public final class Configs {
      */
     public static final String LOG_COLLECTION_ENABLED = "twill.log.collection.enabled";
 
+    /**
+     * The maximum number of FileContext object cached by the FileContextLocationFactory.
+     */
+    public static final String FILE_CONTEXT_CACHE_MAX_SIZE = "twill.file.context.cache.max.size";
+
     private Keys() {
     }
   }
@@ -127,6 +132,11 @@ public final class Configs {
      */
     public static final boolean LOG_COLLECTION_ENABLED = true;
 
+    /**
+     * Default size of the file context cache.
+     */
+    public static final int FILE_CONTEXT_CACHE_MAX_SIZE = 100;
+
 
     private Defaults() {
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/2c42097c/twill-yarn/src/main/java/org/apache/twill/filesystem/FileContextLocationFactory.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/filesystem/FileContextLocationFactory.java b/twill-yarn/src/main/java/org/apache/twill/filesystem/FileContextLocationFactory.java
index b8453bc..29a4db5 100644
--- a/twill-yarn/src/main/java/org/apache/twill/filesystem/FileContextLocationFactory.java
+++ b/twill-yarn/src/main/java/org/apache/twill/filesystem/FileContextLocationFactory.java
@@ -18,13 +18,21 @@
 package org.apache.twill.filesystem;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.util.concurrent.UncheckedExecutionException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.twill.api.Configs;
 
+import java.io.IOException;
 import java.net.URI;
+import java.security.PrivilegedExceptionAction;
 import java.util.Objects;
 
 /**
@@ -33,8 +41,8 @@ import java.util.Objects;
 public class FileContextLocationFactory implements LocationFactory {
 
   private final Configuration configuration;
-  private final FileContext fc;
   private final Path pathBase;
+  private final LoadingCache<UserGroupInformation, FileContext> fileContextCache;
 
   /**
    * Same as {@link #FileContextLocationFactory(Configuration, String) FileContextLocationFactory(configuration, "/")}.
@@ -49,8 +57,28 @@ public class FileContextLocationFactory implements LocationFactory {
    * @param configuration the hadoop configuration
    * @param pathBase base path for all non-absolute location created through this {@link LocationFactory}.
    */
-  public FileContextLocationFactory(Configuration configuration, String pathBase) {
-    this(configuration, createFileContext(configuration), pathBase);
+  public FileContextLocationFactory(final Configuration configuration, String pathBase) {
+    this.configuration = configuration;
+    this.pathBase = new Path(pathBase.startsWith("/") ? pathBase : "/" + pathBase);
+
+    int maxCacheSize = configuration.getInt(Configs.Keys.FILE_CONTEXT_CACHE_MAX_SIZE,
+                                            Configs.Defaults.FILE_CONTEXT_CACHE_MAX_SIZE);
+    this.fileContextCache = CacheBuilder
+      .newBuilder()
+      .weakKeys()
+      .weakValues()
+      .maximumSize(maxCacheSize)
+      .build(new CacheLoader<UserGroupInformation, FileContext>() {
+        @Override
+        public FileContext load(UserGroupInformation ugi) throws Exception {
+          return ugi.doAs(new PrivilegedExceptionAction<FileContext>() {
+            @Override
+            public FileContext run() throws UnsupportedFileSystemException {
+              return FileContext.getFileContext(configuration);
+            }
+          });
+        }
+      });
   }
 
   /**
@@ -59,11 +87,16 @@ public class FileContextLocationFactory implements LocationFactory {
    * @param configuration the hadoop configuration
    * @param fc {@link FileContext} instance created from the given configuration
    * @param pathBase base path for all non-absolute location created through this (@link LocationFactory}.
+   *
+   * @deprecated Use {@link #FileContextLocationFactory(Configuration)}
+   *             or {@link #FileContextLocationFactory(Configuration, String)} instead. The {@link FileContext}
+   *             provided to this method will only be used if the current user calling any methods of this class
+   *             matches with the {@link UserGroupInformation} of the {@link FileContext} instance.
    */
+  @Deprecated
   public FileContextLocationFactory(Configuration configuration, FileContext fc, String pathBase) {
-    this.configuration = configuration;
-    this.fc = fc;
-    this.pathBase = new Path(pathBase.startsWith("/") ? pathBase : "/" + pathBase);
+    this(configuration, pathBase);
+    this.fileContextCache.put(fc.getUgi(), fc);
   }
 
   @Override
@@ -77,12 +110,14 @@ public class FileContextLocationFactory implements LocationFactory {
     } else {
       locationPath = new Path(path);
     }
+    FileContext fc = getFileContext();
     locationPath = locationPath.makeQualified(fc.getDefaultFileSystem().getUri(), pathBase);
     return new FileContextLocation(this, fc, locationPath);
   }
 
   @Override
   public Location create(URI uri) {
+    FileContext fc = getFileContext();
     URI contextURI = fc.getWorkingDirectory().toUri();
     if (Objects.equals(contextURI.getScheme(), uri.getScheme())
       && Objects.equals(contextURI.getAuthority(), uri.getAuthority())) {
@@ -103,16 +138,32 @@ public class FileContextLocationFactory implements LocationFactory {
 
   @Override
   public Location getHomeLocation() {
+    FileContext fc = getFileContext();
     // Fix for TWILL-163. FileContext.getHomeDirectory() uses System.getProperty("user.name") instead of UGI
     return new FileContextLocation(this, fc,
                                    new Path(fc.getHomeDirectory().getParent(), fc.getUgi().getShortUserName()));
   }
 
   /**
-   * Returns the {@link FileContext} used by this {@link LocationFactory}.
+   * Returns the {@link FileContext} for the current user based on {@link UserGroupInformation#getCurrentUser()}.
+   *
+   * @throws IllegalStateException if failed to determine the current user or fail to create the FileContext.
+   * @throws RuntimeException if failed to get the {@link FileContext} object for the current user due to exception
    */
   public FileContext getFileContext() {
-    return fc;
+    try {
+      return fileContextCache.getUnchecked(UserGroupInformation.getCurrentUser());
+    } catch (IOException e) {
+      throw new IllegalStateException("Failed to get current user information", e);
+    } catch (UncheckedExecutionException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof UnsupportedFileSystemException) {
+        String defaultURI = configuration.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+                                              CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT);
+        throw new IllegalStateException("File system with URI '" + defaultURI + "' is not supported", cause);
+      }
+      throw (cause instanceof RuntimeException) ? (RuntimeException) cause : new RuntimeException(cause);
+    }
   }
 
   /**
@@ -121,12 +172,4 @@ public class FileContextLocationFactory implements LocationFactory {
   public Configuration getConfiguration() {
     return configuration;
   }
-
-  private static FileContext createFileContext(Configuration configuration) {
-    try {
-      return FileContext.getFileContext(configuration);
-    } catch (UnsupportedFileSystemException e) {
-      throw Throwables.propagate(e);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/2c42097c/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index d2b53b9..a31265e 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -674,7 +674,8 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
   private static LocationFactory createDefaultLocationFactory(Configuration configuration) {
     try {
       FileContext fc = FileContext.getFileContext(configuration);
-      return new FileContextLocationFactory(configuration, fc, fc.getHomeDirectory().toUri().getPath());
+      String basePath = fc.getHomeDirectory().toUri().getPath();
+      return new FileContextLocationFactory(configuration, basePath);
     } catch (IOException e) {
       throw Throwables.propagate(e);
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/2c42097c/twill-yarn/src/test/java/org/apache/twill/filesystem/FileContextLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/FileContextLocationTest.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/FileContextLocationTest.java
index cbc2781..5629295 100644
--- a/twill-yarn/src/test/java/org/apache/twill/filesystem/FileContextLocationTest.java
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/FileContextLocationTest.java
@@ -17,18 +17,20 @@
  */
 package org.apache.twill.filesystem;
 
-import com.google.common.base.Throwables;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.twill.internal.yarn.YarnUtils;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Test;
 
 import java.io.IOException;
-import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
 
 /**
  *
@@ -36,7 +38,6 @@ import java.security.PrivilegedAction;
 public class FileContextLocationTest extends LocationTestBase {
 
   public static MiniDFSCluster dfsCluster;
-  private static UserGroupInformation testUGI;
 
   @BeforeClass
   public static void init() throws IOException {
@@ -45,9 +46,6 @@ public class FileContextLocationTest extends LocationTestBase {
     dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     // make root world-writable so that we can create all location factories as unprivileged user
     dfsCluster.getFileSystem().setPermission(new Path("/"), FsPermission.valueOf("-rwxrwxrwx"));
-    // to run these tests not as superuser, make sure to use a user name other than the JVM user
-    String userName = System.getProperty("user.name").equals("tester") ? "twiller" : "tester";
-    testUGI = UserGroupInformation.createUserForTesting(userName, new String[] { "testgroup" });
   }
 
   @AfterClass
@@ -56,39 +54,7 @@ public class FileContextLocationTest extends LocationTestBase {
   }
 
   @Override
-  protected String getUserName() {
-    return testUGI.getUserName();
-  }
-
-  @Override
-  protected String getUserGroup(String ignoredGroupName) {
-    return testUGI.getGroupNames()[testUGI.getGroupNames().length - 1];
-  }
-
-  @Override
   protected LocationFactory createLocationFactory(String pathBase) throws Exception {
-    return createLocationFactory(pathBase, testUGI);
-  }
-
-  @Override
-  protected LocationFactory createLocationFactory(final String pathBase, UserGroupInformation ugi) throws Exception {
-    LocationFactory factory = ugi.doAs(new PrivilegedAction<LocationFactory>() {
-      @Override
-      public LocationFactory run() {
-        try {
-          return doCreateLocationFactory(pathBase);
-        } catch (IOException e) {
-          throw Throwables.propagate(e);
-        }
-      }
-    });
-    // make sure the root of the location factory exists and only permits the test user
-    Location root = factory.create("/");
-    root.mkdirs("rwx------");
-    return factory;
-  }
-
-  protected LocationFactory doCreateLocationFactory(String pathBase) throws IOException {
     return new FileContextLocationFactory(dfsCluster.getFileSystem().getConf(), pathBase);
   }
 
@@ -101,4 +67,20 @@ public class FileContextLocationTest extends LocationTestBase {
     }
     return original;
   }
+
+  @Test
+  public void testGetFileContext() throws Exception {
+    final FileContextLocationFactory locationFactory = (FileContextLocationFactory) createLocationFactory("/testGetFC");
+
+    Assert.assertEquals(UserGroupInformation.getCurrentUser(), locationFactory.getFileContext().getUgi());
+
+    UserGroupInformation testUGI = createTestUGI();
+    FileContext fileContext = testUGI.doAs(new PrivilegedExceptionAction<FileContext>() {
+      @Override
+      public FileContext run() throws Exception {
+        return locationFactory.getFileContext();
+      }
+    });
+    Assert.assertEquals(testUGI, fileContext.getUgi());
+  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/2c42097c/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
index a6b98e8..d48fe8e 100644
--- a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
@@ -18,9 +18,9 @@
 package org.apache.twill.filesystem;
 
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.Assert;
 
 import java.io.File;
+import java.io.IOException;
 
 /**
  *
@@ -28,17 +28,6 @@ import java.io.File;
 public class LocalLocationTest extends LocationTestBase {
 
   @Override
-  protected String getUserName() {
-    return System.getProperty("user.name");
-  }
-
-  @Override
-  protected String getUserGroup(String groupName) {
-    String newGroup = System.getProperty("new.group.name");
-    return newGroup != null ? newGroup : groupName;
-  }
-
-  @Override
   protected LocationFactory createLocationFactory(String pathBase) throws Exception {
     File basePath = new File(tmpFolder.newFolder(), pathBase);
     //noinspection ResultOfMethodCallIgnored
@@ -46,13 +35,9 @@ public class LocalLocationTest extends LocationTestBase {
     return new LocalLocationFactory(basePath);
   }
 
-  protected LocationFactory createLocationFactory(String pathBase, UserGroupInformation ugi) throws Exception {
-    return null;
-  }
-
   @Override
-  public void testHomeLocation() throws Exception {
-    // For Local location, UGI won't take an effect.
-    Assert.assertEquals(System.getProperty("user.name"), createLocationFactory("/").getHomeLocation().getName());
+  protected UserGroupInformation createTestUGI() throws IOException {
+    // In local location, UGI is not supported, hence using the current user as the testing ugi.
+    return UserGroupInformation.getCurrentUser();
   }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/2c42097c/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
index 3da5f7a..d42cf37 100644
--- a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
@@ -35,6 +35,8 @@ import java.io.OutputStreamWriter;
 import java.io.Reader;
 import java.io.Writer;
 import java.net.URI;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
 import java.util.List;
 
 /**
@@ -98,16 +100,22 @@ public abstract class LocationTestBase {
 
   @Test
   public void testHomeLocation() throws Exception {
-    LocationFactory locationFactory = createLocationFactory("/");
+    final LocationFactory locationFactory = createLocationFactory("/");
 
-    // Without UGI, the home location should be the same as the user
-    Assert.assertEquals(getUserName(), locationFactory.getHomeLocation().getName());
+    // Without UGI, the home location should be the same as the current user
+    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    Assert.assertEquals(currentUser.getShortUserName(), locationFactory.getHomeLocation().getName());
 
     // With UGI, the home location should be based on the UGI current user
-    UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
-      getUserName() + "1", UserGroupInformation.getCurrentUser().getGroupNames());
-    locationFactory = createLocationFactory("/", ugi);
-    Assert.assertEquals(ugi.getUserName(), locationFactory.getHomeLocation().getName());
+    UserGroupInformation ugi = createTestUGI();
+
+    Location homeLocation = ugi.doAs(new PrivilegedAction<Location>() {
+      @Override
+      public Location run() {
+        return locationFactory.getHomeLocation();
+      }
+    });
+    Assert.assertEquals(ugi.getUserName(), homeLocation.getName());
   }
 
   @Test
@@ -184,163 +192,187 @@ public abstract class LocationTestBase {
 
   @Test
   public void testOwnerGroup() throws Exception {
-    LocationFactory factory = locationFactoryCache.getUnchecked("ownergroup");
-    Location location = factory.create("ogtest");
+    final LocationFactory factory = locationFactoryCache.getUnchecked("ownergroup");
+
+    UserGroupInformation testUGI = createTestUGI();
+    Location location = testUGI.doAs(new PrivilegedExceptionAction<Location>() {
+      @Override
+      public Location run() throws Exception {
+        return factory.create("ogtest");
+      }
+    });
+
     location.createNew();
-    Assert.assertEquals(getUserName(), location.getOwner());
-    String newGroup =  getUserGroup(location.getGroup());
-    location.setGroup(newGroup);
-    Assert.assertEquals(newGroup, location.getGroup());
-  }
+    Assert.assertEquals(testUGI.getUserName(), location.getOwner());
 
-  @Test
-  public void testPermissions() throws IOException {
-    LocationFactory factory = locationFactoryCache.getUnchecked("permission1");
-
-    // Test permissions setting on createNew calls
-    Location location = factory.create("test400-1");
-    Assert.assertTrue(location.createNew("400"));
-    Assert.assertEquals("r--------", location.getPermissions());
-    location = factory.create("test400-2");
-    Assert.assertTrue(location.createNew("r--------"));
-    Assert.assertEquals("r--------", location.getPermissions());
-    Assert.assertFalse(location.createNew("600"));
-
-    location = factory.create("test660-1");
-    Assert.assertTrue(location.createNew("660"));
-    Assert.assertEquals("rw-rw----", location.getPermissions());
-    location = factory.create("test660-2");
-    Assert.assertTrue(location.createNew("rw-rw----"));
-    Assert.assertEquals("rw-rw----", location.getPermissions());
-    Assert.assertFalse(location.createNew("600"));
-
-    location = factory.create("test644-1");
-    Assert.assertTrue(location.createNew("644"));
-    Assert.assertEquals("rw-r--r--", location.getPermissions());
-    location = factory.create("test644-2");
-    Assert.assertTrue(location.createNew("rw-r--r--"));
-    Assert.assertEquals("rw-r--r--", location.getPermissions());
-    Assert.assertFalse(location.createNew("600"));
-
-    // Test permissions setting on getOutputStream calls
-    factory = locationFactoryCache.getUnchecked("permission2");
-
-    location = factory.create("test400-1");
-    location.getOutputStream("400").close();
-    Assert.assertEquals("r--------", location.getPermissions());
-    location = factory.create("test400-2");
-    location.getOutputStream("r--------").close();
-    Assert.assertEquals("r--------", location.getPermissions());
-
-    location = factory.create("test660-1");
-    location.getOutputStream("660").close();
-    Assert.assertEquals("rw-rw----", location.getPermissions());
-    location = factory.create("test660-2");
-    location.getOutputStream("rw-rw----").close();
-    Assert.assertEquals("rw-rw----", location.getPermissions());
-
-    location = factory.create("test644-1");
-    location.getOutputStream("644").close();
-    Assert.assertEquals("rw-r--r--", location.getPermissions());
-    location = factory.create("test644-2");
-    location.getOutputStream("rw-r--r--").close();
-    Assert.assertEquals("rw-r--r--", location.getPermissions());
-
-    // Test permissions setting on setPermission method
-    factory = locationFactoryCache.getUnchecked("permission3");
-
-    // Setting permission on non-existed file should have IOException thrown
-    location = factory.create("somefile");
-    try {
-      location.setPermissions("400");
-      Assert.fail("IOException expected on setting permission on non-existing Location.");
-    } catch (IOException e) {
-      // expected
-    }
+    String group = testUGI.getGroupNames()[0];
 
-    // Create file with read only permission
-    Assert.assertTrue(location.createNew("444"));
-    Assert.assertEquals("r--r--r--", location.getPermissions());
-    // Change the permission to write only
-    location.setPermissions("222");
-    Assert.assertEquals("-w--w--w-", location.getPermissions());
+    location.setGroup(group);
+    Assert.assertEquals(group, location.getGroup());
   }
 
   @Test
-  public void testDirPermissions() throws IOException {
-    LocationFactory factory = locationFactoryCache.getUnchecked("permissionD");
+  public void testPermissions() throws IOException, InterruptedException {
+    createTestUGI().doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        LocationFactory factory = locationFactoryCache.getUnchecked("permission1");
+
+        // Test permissions setting on createNew calls
+        Location location = factory.create("test400-1");
+        Assert.assertTrue(location.createNew("400"));
+        Assert.assertEquals("r--------", location.getPermissions());
+        location = factory.create("test400-2");
+        Assert.assertTrue(location.createNew("r--------"));
+        Assert.assertEquals("r--------", location.getPermissions());
+        Assert.assertFalse(location.createNew("600"));
+
+        location = factory.create("test660-1");
+        Assert.assertTrue(location.createNew("660"));
+        Assert.assertEquals("rw-rw----", location.getPermissions());
+        location = factory.create("test660-2");
+        Assert.assertTrue(location.createNew("rw-rw----"));
+        Assert.assertEquals("rw-rw----", location.getPermissions());
+        Assert.assertFalse(location.createNew("600"));
+
+        location = factory.create("test644-1");
+        Assert.assertTrue(location.createNew("644"));
+        Assert.assertEquals("rw-r--r--", location.getPermissions());
+        location = factory.create("test644-2");
+        Assert.assertTrue(location.createNew("rw-r--r--"));
+        Assert.assertEquals("rw-r--r--", location.getPermissions());
+        Assert.assertFalse(location.createNew("600"));
+
+        // Test permissions setting on getOutputStream calls
+        factory = locationFactoryCache.getUnchecked("permission2");
+
+        location = factory.create("test400-1");
+        location.getOutputStream("400").close();
+        Assert.assertEquals("r--------", location.getPermissions());
+        location = factory.create("test400-2");
+        location.getOutputStream("r--------").close();
+        Assert.assertEquals("r--------", location.getPermissions());
+
+        location = factory.create("test660-1");
+        location.getOutputStream("660").close();
+        Assert.assertEquals("rw-rw----", location.getPermissions());
+        location = factory.create("test660-2");
+        location.getOutputStream("rw-rw----").close();
+        Assert.assertEquals("rw-rw----", location.getPermissions());
+
+        location = factory.create("test644-1");
+        location.getOutputStream("644").close();
+        Assert.assertEquals("rw-r--r--", location.getPermissions());
+        location = factory.create("test644-2");
+        location.getOutputStream("rw-r--r--").close();
+        Assert.assertEquals("rw-r--r--", location.getPermissions());
+
+        // Test permissions setting on setPermission method
+        factory = locationFactoryCache.getUnchecked("permission3");
+
+        // Setting permission on non-existed file should have IOException thrown
+        location = factory.create("somefile");
+        try {
+          location.setPermissions("400");
+          Assert.fail("IOException expected on setting permission on non-existing Location.");
+        } catch (IOException e) {
+          // expected
+        }
+
+        // Create file with read only permission
+        Assert.assertTrue(location.createNew("444"));
+        Assert.assertEquals("r--r--r--", location.getPermissions());
+        // Change the permission to write only
+        location.setPermissions("222");
+        Assert.assertEquals("-w--w--w-", location.getPermissions());
+
+        return null;
+      }
+    });
+  }
 
-    Location location = factory.create("nn");
-    String permissions = "rwxr-x---";
-    location.mkdirs(permissions);
-    Assert.assertTrue(location.exists());
-    Assert.assertTrue(location.isDirectory());
-    Assert.assertEquals(permissions, location.getPermissions());
-
-    permissions = "rwx------";
-    location.setPermissions(permissions);
-    Assert.assertEquals(permissions, location.getPermissions());
-
-    Location child = location.append("p1");
-    Location grandchild = child.append("xx");
-    permissions = "rwx-w--w-";
-    grandchild.mkdirs(permissions);
-    Assert.assertTrue(child.isDirectory());
-    Assert.assertTrue(grandchild.isDirectory());
-    Assert.assertEquals(permissions, child.getPermissions());
-    Assert.assertEquals(permissions, grandchild.getPermissions());
-
-    permissions = "rwx------";
-    child.delete(true);
-    Assert.assertFalse(child.exists());
-    Location textfile = grandchild.append("a.txt");
-    textfile.getOutputStream(permissions).close();
-    Assert.assertTrue(child.isDirectory());
-    Assert.assertTrue(grandchild.isDirectory());
-    Assert.assertFalse(textfile.isDirectory());
-    Assert.assertEquals(permissions, child.getPermissions());
-    Assert.assertEquals(permissions, grandchild.getPermissions());
-    Assert.assertEquals(correctFilePermissions(permissions), textfile.getPermissions());
-
-    // mkdirs of existing file
-    Location file = factory.create("existingfile");
-    Assert.assertTrue(file.createNew("rwx------"));
-    Assert.assertFalse(file.mkdirs());
-    Assert.assertFalse(file.mkdirs("rwxrwx---"));
-
-    // mkdirs where parent is existing file
-    file = file.append("newdir");
-    Assert.assertFalse(file.mkdirs());
-    Assert.assertFalse(file.mkdirs("rwxrwx---"));
-
-    // mkdirs of existing directory
-    Location dir = factory.create("existingdir");
-    Assert.assertTrue(dir.mkdirs());
-    Assert.assertFalse(dir.mkdirs());
-    Assert.assertFalse(dir.mkdirs("rwxrwx---"));
-
-    // mkdirs for existing parent with different permissions -> should not change
-    dir.setPermissions("rwx------");
-    Assert.assertEquals("rwx------", dir.getPermissions());
-    Location newdir = dir.append("newdir");
-    Assert.assertTrue(newdir.mkdirs("rwxrwx---"));
-    Assert.assertEquals("rwxrwx---", newdir.getPermissions());
-    Assert.assertEquals("rwx------", dir.getPermissions());
-
-    // mkdirs whithout permission for parent
-    Assert.assertTrue(newdir.delete(true));
-    dir.setPermissions("r-x------");
-    Assert.assertEquals("r-x------", dir.getPermissions());
-    try {
-      Assert.assertFalse(newdir.mkdirs());
-    } catch (AccessControlException e) {
-      // expected
-    }
-    try {
-      Assert.assertFalse(newdir.mkdirs("rwxrwx---"));
-    } catch (AccessControlException e) {
-      // expected
-    }
+  @Test
+  public void testDirPermissions() throws IOException, InterruptedException {
+    createTestUGI().doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+
+        LocationFactory factory = locationFactoryCache.getUnchecked("permissionD");
+
+        Location location = factory.create("base");
+        String permissions = "rwxr-x---";
+        location.mkdirs(permissions);
+        Assert.assertTrue(location.exists());
+        Assert.assertTrue(location.isDirectory());
+        Assert.assertEquals(permissions, location.getPermissions());
+
+        permissions = "rwx------";
+        location.setPermissions(permissions);
+        Assert.assertEquals(permissions, location.getPermissions());
+
+        Location child = location.append("child");
+        Location grandchild = child.append("grandchild");
+        permissions = "rwx-w--w-";
+        grandchild.mkdirs(permissions);
+        Assert.assertTrue(child.isDirectory());
+        Assert.assertTrue(grandchild.isDirectory());
+        Assert.assertEquals(permissions, child.getPermissions());
+        Assert.assertEquals(permissions, grandchild.getPermissions());
+
+        permissions = "rwx------";
+        child.delete(true);
+        Assert.assertFalse(child.exists());
+        Location textfile = grandchild.append("a.txt");
+        textfile.getOutputStream(permissions).close();
+        Assert.assertTrue(child.isDirectory());
+        Assert.assertTrue(grandchild.isDirectory());
+        Assert.assertFalse(textfile.isDirectory());
+        Assert.assertEquals(permissions, child.getPermissions());
+        Assert.assertEquals(permissions, grandchild.getPermissions());
+        Assert.assertEquals(correctFilePermissions(permissions), textfile.getPermissions());
+
+        // mkdirs of existing file
+        Location file = factory.create("existingfile");
+        Assert.assertTrue(file.createNew("rwx------"));
+        Assert.assertFalse(file.mkdirs());
+        Assert.assertFalse(file.mkdirs("rwxrwx---"));
+
+        // mkdirs where parent is existing file
+        file = file.append("newdir");
+        Assert.assertFalse(file.mkdirs());
+        Assert.assertFalse(file.mkdirs("rwxrwx---"));
+
+        // mkdirs of existing directory
+        Location dir = factory.create("existingdir");
+        Assert.assertTrue(dir.mkdirs());
+        Assert.assertFalse(dir.mkdirs());
+        Assert.assertFalse(dir.mkdirs("rwxrwx---"));
+
+        // mkdirs for existing parent with different permissions -> should not change
+        dir.setPermissions("rwx------");
+        Assert.assertEquals("rwx------", dir.getPermissions());
+        Location newdir = dir.append("newdir");
+        Assert.assertTrue(newdir.mkdirs("rwxrwx---"));
+        Assert.assertEquals("rwxrwx---", newdir.getPermissions());
+        Assert.assertEquals("rwx------", dir.getPermissions());
+
+        // mkdirs whithout permission for parent
+        Assert.assertTrue(newdir.delete(true));
+        dir.setPermissions("r-x------");
+        Assert.assertEquals("r-x------", dir.getPermissions());
+        try {
+          Assert.assertFalse(newdir.mkdirs());
+        } catch (AccessControlException e) {
+          // expected
+        }
+        try {
+          Assert.assertFalse(newdir.mkdirs("rwxrwx---"));
+        } catch (AccessControlException e) {
+          // expected
+        }
+        return null;
+      }
+    });
   }
 
   /**
@@ -349,26 +381,15 @@ public abstract class LocationTestBase {
   protected abstract LocationFactory createLocationFactory(String pathBase) throws Exception;
 
   /**
-   * Create a location factory rooted at a given path, for the given UGI.
-   */
-  protected abstract LocationFactory createLocationFactory(String pathBase, UserGroupInformation ugi) throws Exception;
-
-  /**
-   * Get the user name used for {@link #createLocationFactory(String)}.
-   */
-  protected abstract String getUserName();
-
-  /**
-   * Given the group name of a location, return a valid group name to test changing the location's group.
-   * If no suitable group name is known, the passed-in group name can be returned.
-   */
-  protected abstract String getUserGroup(String groupName);
-
-  /**
    * Some older versions of Hadoop always strip the execute permission from files (but keep it for directories).
    * This allows subclasses to correct the expected file permissions, based on the Hadoop version (if any).
    */
   protected String correctFilePermissions(String expectedFilePermissions) {
     return expectedFilePermissions; // unchanged by default
   }
+
+  protected UserGroupInformation createTestUGI() throws IOException {
+    String userName = System.getProperty("user.name").equals("tester") ? "twiller" : "tester";
+    return UserGroupInformation.createUserForTesting(userName, new String[] { "testgroup" });
+  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/2c42097c/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java b/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
index a141176..04902d5 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/TwillTester.java
@@ -207,7 +207,8 @@ public class TwillTester extends ExternalResource {
   public LocationFactory createLocationFactory() {
     try {
       FileContext fc = FileContext.getFileContext(config);
-      return new FileContextLocationFactory(config, fc, fc.getHomeDirectory().toUri().getPath());
+      String basePath = fc.getHomeDirectory().toUri().getPath();
+      return new FileContextLocationFactory(config, basePath);
     } catch (IOException e) {
       throw Throwables.propagate(e);
     }


[07/24] twill git commit: (TWILL-122) Allow disabling log collection

Posted by ch...@apache.org.
(TWILL-122) Allow disabling log collection

- Introduced a new configuration twill.log.collection.enabled for
  turning off log collection
- Refactor YarnTwillController and related class hierarchy to not
  starting Kafka client when log collection is disabled
- Added Kafka zk connection string information in AM live node data
- Refactor KafkaAppender and ServiceMain configureLogger
  - Log to StatusManager instead of Logger to avoid recursive logging
  - Instead of resetting logback configuration, directly instantiate and
    add the Kafka log appender to the logging context.
- Refactor ServiceMain, ApplicationMasterMain and TwillContainerMain to
  simplify ZK Connection string construction

This closes #40 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/3045b91b
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/3045b91b
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/3045b91b

Branch: refs/heads/site
Commit: 3045b91b0367448db0dd3146db3bd34c107eb4c5
Parents: e154bfe
Author: Terence Yim <ch...@apache.org>
Authored: Mon Mar 20 14:42:46 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Thu Mar 23 15:13:47 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/twill/api/Configs.java |  10 ++
 .../twill/internal/AbstractTwillController.java |  36 +++++-
 .../java/org/apache/twill/internal/EnvKeys.java |   2 -
 .../internal/TwillRuntimeSpecification.java     |  26 +++-
 .../json/TwillRuntimeSpecificationCodec.java    |   8 +-
 .../twill/internal/logging/KafkaAppender.java   |  30 ++---
 .../apache/twill/internal/ControllerTest.java   |  11 +-
 .../org/apache/twill/internal/ServiceMain.java  | 120 ++++++-------------
 .../ApplicationMasterLiveNodeData.java          |  15 ++-
 .../appmaster/ApplicationMasterMain.java        |  94 +++++++++++----
 .../appmaster/ApplicationMasterService.java     |   9 +-
 .../internal/container/TwillContainerMain.java  |  27 ++---
 .../apache/twill/yarn/YarnTwillController.java  |   8 +-
 .../twill/yarn/YarnTwillControllerFactory.java  |   2 +-
 .../apache/twill/yarn/YarnTwillPreparer.java    |   9 +-
 .../twill/yarn/YarnTwillRunnerService.java      |   3 +-
 .../apache/twill/yarn/LogHandlerTestRun.java    |  44 +++++++
 17 files changed, 279 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-api/src/main/java/org/apache/twill/api/Configs.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/Configs.java b/twill-api/src/main/java/org/apache/twill/api/Configs.java
index 570eafe..1447a37 100644
--- a/twill-api/src/main/java/org/apache/twill/api/Configs.java
+++ b/twill-api/src/main/java/org/apache/twill/api/Configs.java
@@ -78,6 +78,11 @@ public final class Configs {
      */
     public static final String YARN_AM_RESERVED_MEMORY_MB = "twill.yarn.am.reserved.memory.mb";
 
+    /**
+     * Setting for enabling log collection.
+     */
+    public static final String LOG_COLLECTION_ENABLED = "twill.log.collection.enabled";
+
     private Keys() {
     }
   }
@@ -117,6 +122,11 @@ public final class Configs {
      */
     public static final int YARN_AM_RESERVED_MEMORY_MB = 150;
 
+    /**
+     * Default to enable log collection.
+     */
+    public static final boolean LOG_COLLECTION_ENABLED = true;
+
 
     private Defaults() {
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
index 212411f..e49a2ad 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
@@ -71,21 +71,37 @@ public abstract class AbstractTwillController extends AbstractZKServiceControlle
   private static final Logger LOG = LoggerFactory.getLogger(AbstractTwillController.class);
   private static final Gson GSON = new Gson();
 
+  private final String appName;
+  private final RunId runId;
   private final Queue<LogHandler> logHandlers;
   private final KafkaClientService kafkaClient;
   private ZKDiscoveryService discoveryServiceClient;
   private Cancellable logCancellable;
 
-  public AbstractTwillController(RunId runId, ZKClient zkClient, Iterable<LogHandler> logHandlers) {
+  public AbstractTwillController(String appName, RunId runId, ZKClient zkClient, boolean logCollectionEnabled,
+                                 Iterable<LogHandler> logHandlers) {
     super(runId, zkClient);
+    this.appName = appName;
+    this.runId = runId;
     this.logHandlers = new ConcurrentLinkedQueue<>();
-    this.kafkaClient = new ZKKafkaClientService(ZKClients.namespace(zkClient, "/" + runId.getId() + "/kafka"));
-    Iterables.addAll(this.logHandlers, logHandlers);
+
+    // When addressing TWILL-147, need to check if the given ZKClient is
+    // actually used by the Kafka used for log collection
+    if (logCollectionEnabled) {
+      this.kafkaClient = new ZKKafkaClientService(ZKClients.namespace(zkClient, "/" + runId.getId() + "/kafka"));
+      Iterables.addAll(this.logHandlers, logHandlers);
+    } else {
+      this.kafkaClient = null;
+      if (!Iterables.isEmpty(logHandlers)) {
+        LOG.warn("Log collection is disabled for application {} with runId {}. " +
+                   "Adding log handler won't get any logs.", appName, runId);
+      }
+    }
   }
 
   @Override
   protected synchronized void doStartUp() {
-    if (!logHandlers.isEmpty()) {
+    if (kafkaClient != null && !logHandlers.isEmpty()) {
       kafkaClient.startAndWait();
       logCancellable = kafkaClient.getConsumer().prepare()
                                   .addFromBeginning(Constants.LOG_TOPIC, 0)
@@ -101,12 +117,20 @@ public abstract class AbstractTwillController extends AbstractZKServiceControlle
     if (discoveryServiceClient != null) {
       discoveryServiceClient.close();
     }
-    // Safe to call stop no matter when state the KafkaClientService is in.
-    kafkaClient.stopAndWait();
+    if (kafkaClient != null) {
+      // Safe to call stop no matter what state the KafkaClientService is in.
+      kafkaClient.stopAndWait();
+    }
   }
 
   @Override
   public final synchronized void addLogHandler(LogHandler handler) {
+    if (kafkaClient == null) {
+      LOG.warn("Log collection is disabled for application {} with runId {}. " +
+                 "Adding log handler won't get any logs.", appName, runId);
+      return;
+    }
+
     logHandlers.add(handler);
     if (logHandlers.size() == 1) {
       kafkaClient.startAndWait();

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-core/src/main/java/org/apache/twill/internal/EnvKeys.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/EnvKeys.java b/twill-core/src/main/java/org/apache/twill/internal/EnvKeys.java
index 6948f80..8f37f5e 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/EnvKeys.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/EnvKeys.java
@@ -31,8 +31,6 @@ public final class EnvKeys {
    */
   public static final String TWILL_RUNNABLE_NAME = "TWILL_RUNNABLE_NAME";
 
-  public static final String TWILL_LOG_KAFKA_ZK = "TWILL_LOG_KAFKA_ZK";
-
   public static final String YARN_APP_ID = "YARN_APP_ID";
   public static final String YARN_APP_ID_CLUSTER_TIME = "YARN_APP_ID_CLUSTER_TIME";
   public static final String YARN_APP_ID_STR = "YARN_APP_ID_STR";

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-core/src/main/java/org/apache/twill/internal/TwillRuntimeSpecification.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillRuntimeSpecification.java b/twill-core/src/main/java/org/apache/twill/internal/TwillRuntimeSpecification.java
index 965b203..831c831 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillRuntimeSpecification.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillRuntimeSpecification.java
@@ -41,13 +41,14 @@ public class TwillRuntimeSpecification {
   private final String rmSchedulerAddr;
   private final Map<String, Map<String, String>> logLevels;
   private final Map<String, Integer> maxRetries;
-  private double minHeapRatio;
+  private final double minHeapRatio;
+  private final boolean logCollectionEnabled;
 
   public TwillRuntimeSpecification(TwillSpecification twillSpecification, String fsUser, URI twillAppDir,
                                    String zkConnectStr, RunId twillRunId, String twillAppName,
                                    int reservedMemory, @Nullable String rmSchedulerAddr,
                                    Map<String, Map<String, String>> logLevels, Map<String, Integer> maxRetries,
-                                   double minHeapRatio) {
+                                   double minHeapRatio, boolean logCollectionEnabled) {
     this.twillSpecification = twillSpecification;
     this.fsUser = fsUser;
     this.twillAppDir = twillAppDir;
@@ -59,6 +60,7 @@ public class TwillRuntimeSpecification {
     this.logLevels = logLevels;
     this.maxRetries = maxRetries;
     this.minHeapRatio = minHeapRatio;
+    this.logCollectionEnabled = logCollectionEnabled;
   }
 
   public TwillSpecification getTwillSpecification() {
@@ -93,6 +95,13 @@ public class TwillRuntimeSpecification {
     return minHeapRatio;
   }
 
+  /**
+   * Returns whether log collection is enabled.
+   */
+  public boolean isLogCollectionEnabled() {
+    return logCollectionEnabled;
+  }
+
   @Nullable
   public String getRmSchedulerAddr() {
     return rmSchedulerAddr;
@@ -105,4 +114,17 @@ public class TwillRuntimeSpecification {
   public Map<String, Integer> getMaxRetries() {
     return maxRetries;
   }
+
+  /**
+   * Returns the ZK connection string for the Kafka used for log collections,
+   * or {@code null} if log collection is disabled.
+   */
+  @Nullable
+  public String getKafkaZKConnect() {
+    if (!isLogCollectionEnabled()) {
+      return null;
+    }
+    // When addressing TWILL-147, a field can be introduced to carry this value.
+    return String.format("%s/%s/%s/kafka", getZkConnectStr(), getTwillAppName(), getTwillAppRunId());
+  }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRuntimeSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRuntimeSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRuntimeSpecificationCodec.java
index f14fc18..5ff05e8 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRuntimeSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRuntimeSpecificationCodec.java
@@ -25,7 +25,6 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
-import org.apache.twill.api.Configs;
 import org.apache.twill.api.TwillSpecification;
 import org.apache.twill.internal.RunIds;
 import org.apache.twill.internal.TwillRuntimeSpecification;
@@ -51,6 +50,7 @@ final class TwillRuntimeSpecificationCodec implements JsonSerializer<TwillRuntim
   private static final String TWILL_SPEC = "twillSpecification";
   private static final String LOG_LEVELS = "logLevels";
   private static final String MAX_RETRIES = "maxRetries";
+  private static final String LOG_COLLECTION_ENABLED = "logCollectionEnabled";
 
   @Override
   public JsonElement serialize(TwillRuntimeSpecification src, Type typeOfSrc, JsonSerializationContext context) {
@@ -71,6 +71,7 @@ final class TwillRuntimeSpecificationCodec implements JsonSerializer<TwillRuntim
              context.serialize(src.getLogLevels(), new TypeToken<Map<String, Map<String, String>>>() { }.getType()));
     json.add(MAX_RETRIES,
              context.serialize(src.getMaxRetries(), new TypeToken<Map<String, Integer>>() { }.getType()));
+    json.addProperty(LOG_COLLECTION_ENABLED, src.isLogCollectionEnabled());
 
     return json;
   }
@@ -98,8 +99,7 @@ final class TwillRuntimeSpecificationCodec implements JsonSerializer<TwillRuntim
                                          jsonObj.get(RM_SCHEDULER_ADDR).getAsString() : null,
                                          logLevels,
                                          maxRetries,
-                                         jsonObj.has(HEAP_RESERVED_MIN_RATIO) ?
-                                         jsonObj.get(HEAP_RESERVED_MIN_RATIO).getAsDouble()
-                                         : Configs.Defaults.HEAP_RESERVED_MIN_RATIO);
+                                         jsonObj.get(HEAP_RESERVED_MIN_RATIO).getAsDouble(),
+                                         jsonObj.get(LOG_COLLECTION_ENABLED).getAsBoolean());
   }
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
index 90d7415..493c4ca 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
@@ -18,7 +18,8 @@
 package org.apache.twill.internal.logging;
 
 import ch.qos.logback.classic.spi.ILoggingEvent;
-import ch.qos.logback.core.AppenderBase;
+import ch.qos.logback.core.Appender;
+import ch.qos.logback.core.UnsynchronizedAppenderBase;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
@@ -44,8 +45,6 @@ import org.apache.twill.zookeeper.RetryStrategies;
 import org.apache.twill.zookeeper.ZKClientService;
 import org.apache.twill.zookeeper.ZKClientServices;
 import org.apache.twill.zookeeper.ZKClients;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.nio.ByteBuffer;
 import java.util.Collection;
@@ -61,11 +60,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
- *
+ * A logback {@link Appender} for writing log events to Kafka.
  */
-public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
+public final class KafkaAppender extends UnsynchronizedAppenderBase<ILoggingEvent> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(KafkaAppender.class);
+  private static final String PUBLISH_THREAD_NAME = "kafka-logger";
 
   private final AtomicReference<KafkaPublisher.Preparer> publisher;
   private final Runnable flushTask;
@@ -146,7 +145,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
     Preconditions.checkNotNull(zkConnectStr);
 
     eventConverter = new LogEventConverter(hostname, runnableName);
-    scheduler = Executors.newSingleThreadScheduledExecutor(Threads.createDaemonThreadFactory("kafka-logger"));
+    scheduler = Executors.newSingleThreadScheduledExecutor(Threads.createDaemonThreadFactory(PUBLISH_THREAD_NAME));
 
     zkClientService = ZKClientServices.delegate(
       ZKClients.reWatchOnExpire(
@@ -162,14 +161,14 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
           Preconditions.checkState(Futures.getUnchecked(future) == Service.State.RUNNING,
                                    "Service is not running.");
         }
-        LOG.info("Kafka client started: " + zkConnectStr);
+        addInfo("Kafka client started: " + zkConnectStr);
         scheduler.scheduleWithFixedDelay(flushTask, 0, flushPeriod, TimeUnit.MILLISECONDS);
       }
 
       @Override
       public void onFailure(Throwable t) {
         // Fail to talk to kafka. Other than logging, what can be done?
-        LOG.error("Failed to start kafka appender.", t);
+        addError("Failed to start kafka appender.", t);
       }
     }, Threads.SAME_THREAD_EXECUTOR);
 
@@ -187,7 +186,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
     try {
       scheduler.submit(flushTask).get(2, TimeUnit.SECONDS);
     } catch (Exception e) {
-      LOG.error("Failed to force log flush in 2 seconds.", e);
+      addError("Failed to force log flush in 2 seconds.", e);
     }
   }
 
@@ -229,7 +228,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
           bufferedSize.addAndGet(-published);
           return published;
         } catch (ExecutionException e) {
-          LOG.error("Failed to publish logs to Kafka.", e);
+          addError("Failed to publish logs to Kafka.", e);
           TimeUnit.NANOSECONDS.sleep(backOffTime);
           publishTimeout -= stopwatch.elapsedTime(timeoutUnit);
           stopwatch.reset();
@@ -237,7 +236,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
         }
       } while (publishTimeout > 0);
     } catch (InterruptedException e) {
-      LOG.warn("Logs publish to Kafka interrupted.", e);
+      addWarn("Logs publish to Kafka interrupted.", e);
     }
     return 0;
   }
@@ -277,12 +276,9 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
       @Override
       public void run() {
         try {
-          int published = publishLogs(2L, TimeUnit.SECONDS);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Published {} log messages to Kafka.", published);
-          }
+          publishLogs(2L, TimeUnit.SECONDS);
         } catch (Exception e) {
-          LOG.error("Failed to push logs to Kafka. Log entries dropped.", e);
+          addError("Failed to push logs to Kafka. Log entries dropped.", e);
         }
       }
     };

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java b/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
index a5cb04b..c37cd00 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
@@ -62,7 +62,7 @@ public class ControllerTest {
       Service service = createService(zkClientService, runId);
       service.startAndWait();
 
-      TwillController controller = getController(zkClientService, runId);
+      TwillController controller = getController(zkClientService, "testController", runId);
       controller.sendCommand(Command.Builder.of("test").build()).get(2, TimeUnit.SECONDS);
       controller.terminate().get(2, TimeUnit.SECONDS);
 
@@ -96,7 +96,7 @@ public class ControllerTest {
       zkClientService.startAndWait();
 
       final CountDownLatch runLatch = new CountDownLatch(1);
-      TwillController controller = getController(zkClientService, runId);
+      TwillController controller = getController(zkClientService, "testControllerBefore", runId);
       controller.onRunning(new Runnable() {
         @Override
         public void run() {
@@ -140,7 +140,7 @@ public class ControllerTest {
       service.startAndWait();
 
       final CountDownLatch runLatch = new CountDownLatch(1);
-      TwillController controller = getController(zkClientService, runId);
+      TwillController controller = getController(zkClientService, "testControllerListener", runId);
       controller.onRunning(new Runnable() {
         @Override
         public void run() {
@@ -185,8 +185,9 @@ public class ControllerTest {
     };
   }
 
-  private TwillController getController(ZKClient zkClient, RunId runId) {
-    AbstractTwillController controller = new AbstractTwillController(runId, zkClient, ImmutableList.<LogHandler>of()) {
+  private TwillController getController(ZKClient zkClient, String appName, RunId runId) {
+    AbstractTwillController controller = new AbstractTwillController(appName, runId,
+                                                                     zkClient, false, ImmutableList.<LogHandler>of()) {
 
       @Override
       public void kill() {

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
index 727435c..0bf07e0 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
@@ -18,9 +18,6 @@
 package org.apache.twill.internal;
 
 import ch.qos.logback.classic.LoggerContext;
-import ch.qos.logback.classic.joran.JoranConfigurator;
-import ch.qos.logback.classic.util.ContextInitializer;
-import ch.qos.logback.core.joran.spi.JoranException;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.common.util.concurrent.Futures;
@@ -45,14 +42,13 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.ILoggerFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.xml.sax.InputSource;
 
 import java.io.File;
-import java.io.StringReader;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 
 /**
  * Class for main method that starts a service.
@@ -70,7 +66,10 @@ public abstract class ServiceMain {
 
   protected final void doMain(final Service mainService,
                               Service...prerequisites) throws ExecutionException, InterruptedException {
-    configureLogger();
+    // Only configure the log collection if it is enabled.
+    if (getTwillRuntimeSpecification().isLogCollectionEnabled()) {
+      configureLogger();
+    }
 
     Service requiredServices = new CompositeService(prerequisites);
     Runtime.getRuntime().addShutdownHook(new Thread() {
@@ -121,14 +120,21 @@ public abstract class ServiceMain {
 
   protected abstract String getHostname();
 
-  protected abstract String getKafkaZKConnect();
+  /**
+   * Returns the {@link TwillRuntimeSpecification} for this application.
+   */
+  protected abstract TwillRuntimeSpecification getTwillRuntimeSpecification();
 
+  /**
+   * Returns the name of the runnable that this running inside this process.
+   */
+  @Nullable
   protected abstract String getRunnableName();
 
   /**
    * Returns the {@link Location} for the application based on the app directory.
    */
-  protected static Location createAppLocation(final Configuration conf, String fsUser, final URI appDir) {
+  protected final Location createAppLocation(final Configuration conf, String fsUser, final URI appDir) {
     // Note: It's a little bit hacky based on the uri schema to create the LocationFactory, refactor it later.
 
     try {
@@ -168,16 +174,19 @@ public abstract class ServiceMain {
   /**
    * Creates a {@link ZKClientService}.
    */
-  protected static ZKClientService createZKClient(String zkConnectStr, String appName) {
+  protected final ZKClientService createZKClient() {
+    TwillRuntimeSpecification twillRuntimeSpec = getTwillRuntimeSpecification();
+
     return ZKClientServices.delegate(
       ZKClients.namespace(
         ZKClients.reWatchOnExpire(
           ZKClients.retryOnFailure(
-            ZKClientService.Builder.of(zkConnectStr).build(),
+            ZKClientService.Builder.of(twillRuntimeSpec.getZkConnectStr()).build(),
             RetryStrategies.fixDelay(1, TimeUnit.SECONDS)
           )
-        ), "/" + appName
-      ));
+        ), "/" + twillRuntimeSpec.getTwillAppName()
+      )
+    );
   }
 
   private void configureLogger() {
@@ -188,84 +197,23 @@ public abstract class ServiceMain {
     }
 
     LoggerContext context = (LoggerContext) loggerFactory;
-    context.reset();
-    JoranConfigurator configurator = new JoranConfigurator();
-    configurator.setContext(context);
-
-    try {
-      File twillLogback = new File(Constants.Files.RUNTIME_CONFIG_JAR, Constants.Files.LOGBACK_TEMPLATE);
-      if (twillLogback.exists()) {
-        configurator.doConfigure(twillLogback);
-      }
-      new ContextInitializer(context).autoConfig();
-    } catch (JoranException e) {
-      throw Throwables.propagate(e);
-    }
-    doConfigure(configurator, getLogConfig(getLoggerLevel(context.getLogger(Logger.ROOT_LOGGER_NAME))));
-  }
-
-  private void doConfigure(JoranConfigurator configurator, String config) {
-    try {
-      configurator.doConfigure(new InputSource(new StringReader(config)));
-    } catch (Exception e) {
-      throw Throwables.propagate(e);
-    }
-  }
-
-  private String getLogConfig(String rootLevel) {
-    return
-      "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n" +
-      "<configuration>\n" +
-      "    <appender name=\"KAFKA\" class=\"" + KafkaAppender.class.getName() + "\">\n" +
-      "        <topic>" + Constants.LOG_TOPIC + "</topic>\n" +
-      "        <hostname>" + getHostname() + "</hostname>\n" +
-      "        <zookeeper>" + getKafkaZKConnect() + "</zookeeper>\n" +
-      appendRunnable() +
-      "    </appender>\n" +
-      "    <logger name=\"org.apache.twill.internal.logging\" additivity=\"false\" />\n" +
-      "    <root level=\"" + rootLevel + "\">\n" +
-      "        <appender-ref ref=\"KAFKA\"/>\n" +
-      "    </root>\n" +
-      "</configuration>";
-  }
-
 
-  private String appendRunnable() {
-    // RunnableName for AM is null, so append runnable name to log config only if the name is not null.
-    if (getRunnableName() == null) {
-     return "";
-    } else {
-      return "        <runnableName>" + getRunnableName() + "</runnableName>\n";
+    // Attach the KafkaAppender to the root logger
+    KafkaAppender kafkaAppender = new KafkaAppender();
+    kafkaAppender.setName("KAFKA");
+    kafkaAppender.setTopic(Constants.LOG_TOPIC);
+    kafkaAppender.setHostname(getHostname());
+    // The Kafka ZK Connection shouldn't be null as this method only get called if log collection is enabled
+    kafkaAppender.setZookeeper(getTwillRuntimeSpecification().getKafkaZKConnect());
+    String runnableName = getRunnableName();
+    if (runnableName != null) {
+      kafkaAppender.setRunnableName(runnableName);
     }
-  }
 
-  /**
-   * Return the right log level for the service.
-   *
-   * @param logger the {@link Logger} instance of the service context.
-   * @return String of log level based on {@code slf4j} log levels.
-   */
-  private String getLoggerLevel(Logger logger) {
-    if (logger instanceof ch.qos.logback.classic.Logger) {
-      return ((ch.qos.logback.classic.Logger) logger).getLevel().toString();
-    }
+    kafkaAppender.setContext(context);
+    kafkaAppender.start();
 
-    if (logger.isTraceEnabled()) {
-      return "TRACE";
-    }
-    if (logger.isDebugEnabled()) {
-      return "DEBUG";
-    }
-    if (logger.isInfoEnabled()) {
-      return "INFO";
-    }
-    if (logger.isWarnEnabled()) {
-      return "WARN";
-    }
-    if (logger.isErrorEnabled()) {
-      return "ERROR";
-    }
-    return "OFF";
+    context.getLogger(ch.qos.logback.classic.Logger.ROOT_LOGGER_NAME).addAppender(kafkaAppender);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterLiveNodeData.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterLiveNodeData.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterLiveNodeData.java
index dd4d946..a38a163 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterLiveNodeData.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterLiveNodeData.java
@@ -20,6 +20,7 @@ package org.apache.twill.internal.appmaster;
 import org.apache.twill.api.LocalFile;
 
 import java.util.List;
+import javax.annotation.Nullable;
 
 /**
  * Represents data being stored in the live node of the application master.
@@ -30,13 +31,16 @@ public final class ApplicationMasterLiveNodeData {
   private final long appIdClusterTime;
   private final String containerId;
   private final List<LocalFile> localFiles;
+  private final String kafkaZKConnect;
 
   public ApplicationMasterLiveNodeData(int appId, long appIdClusterTime,
-                                       String containerId, List<LocalFile> localFiles) {
+                                       String containerId, List<LocalFile> localFiles,
+                                       @Nullable String kafkaZKConnect) {
     this.appId = appId;
     this.appIdClusterTime = appIdClusterTime;
     this.containerId = containerId;
     this.localFiles = localFiles;
+    this.kafkaZKConnect = kafkaZKConnect;
   }
 
   public int getAppId() {
@@ -55,6 +59,15 @@ public final class ApplicationMasterLiveNodeData {
     return localFiles;
   }
 
+  /**
+   * @return the Kafka ZK connection string for the Kafka used for log collection;
+   *         if log collection is turned off, a {@code null} value will be returned.
+   */
+  @Nullable
+  public String getKafkaZKConnect() {
+    return kafkaZKConnect;
+  }
+
   @Override
   public String toString() {
     return "ApplicationMasterLiveNodeData{" +

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
index 8dd5046..81c61ac 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
@@ -17,14 +17,15 @@
  */
 package org.apache.twill.internal.appmaster;
 
+import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.Service;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.twill.api.RunId;
 import org.apache.twill.internal.Constants;
-import org.apache.twill.internal.EnvKeys;
 import org.apache.twill.internal.ServiceMain;
 import org.apache.twill.internal.TwillRuntimeSpecification;
 import org.apache.twill.internal.json.TwillRuntimeSpecificationAdapter;
@@ -45,21 +46,20 @@ import java.io.File;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 
 /**
  * Main class for launching {@link ApplicationMasterService}.
  */
 public final class ApplicationMasterMain extends ServiceMain {
 
-  private final String kafkaZKConnect;
-
-  private ApplicationMasterMain(String kafkaZKConnect) {
-    this.kafkaZKConnect = kafkaZKConnect;
-  }
+  private static final Logger LOG = LoggerFactory.getLogger(ApplicationMasterMain.class);
+  private final TwillRuntimeSpecification twillRuntimeSpec;
 
   /**
    * Starts the application master.
@@ -67,10 +67,18 @@ public final class ApplicationMasterMain extends ServiceMain {
   public static void main(String[] args) throws Exception {
     File twillSpec = new File(Constants.Files.RUNTIME_CONFIG_JAR, Constants.Files.TWILL_SPEC);
     TwillRuntimeSpecification twillRuntimeSpec = TwillRuntimeSpecificationAdapter.create().fromJson(twillSpec);
-    String zkConnect = twillRuntimeSpec.getZkConnectStr();
+
+    new ApplicationMasterMain(twillRuntimeSpec).doMain();
+  }
+
+  private ApplicationMasterMain(TwillRuntimeSpecification twillRuntimeSpec) {
+    this.twillRuntimeSpec = twillRuntimeSpec;
+  }
+
+  private void doMain() throws Exception {
     RunId runId = twillRuntimeSpec.getTwillAppRunId();
 
-    ZKClientService zkClientService = createZKClient(zkConnect, twillRuntimeSpec.getTwillAppName());
+    ZKClientService zkClientService = createZKClient();
     Configuration conf = new YarnConfiguration(new HdfsConfiguration(new Configuration()));
     setRMSchedulerAddress(conf, twillRuntimeSpec.getRmSchedulerAddr());
 
@@ -81,13 +89,22 @@ public final class ApplicationMasterMain extends ServiceMain {
                                                      twillRuntimeSpec.getTwillAppDir()));
     TrackerService trackerService = new TrackerService(service);
 
-    new ApplicationMasterMain(service.getKafkaZKConnect())
+    List<Service> prerequisites = Lists.newArrayList(
+      new YarnAMClientService(amClient, trackerService),
+      zkClientService,
+      new AppMasterTwillZKPathService(zkClientService, runId)
+    );
+
+    if (twillRuntimeSpec.isLogCollectionEnabled()) {
+      prerequisites.add(new ApplicationKafkaService(zkClientService, twillRuntimeSpec.getKafkaZKConnect()));
+    } else {
+      LOG.info("Log collection through kafka disabled");
+    }
+
+    new ApplicationMasterMain(twillRuntimeSpec)
       .doMain(
         service,
-        new YarnAMClientService(amClient, trackerService),
-        zkClientService,
-        new AppMasterTwillZKPathService(zkClientService, runId),
-        new ApplicationKafkaService(zkClientService, runId)
+        prerequisites.toArray(new Service[prerequisites.size()])
       );
   }
 
@@ -117,13 +134,15 @@ public final class ApplicationMasterMain extends ServiceMain {
   }
 
   @Override
-  protected String getKafkaZKConnect() {
-    return kafkaZKConnect;
+  protected TwillRuntimeSpecification getTwillRuntimeSpecification() {
+    return twillRuntimeSpec;
   }
 
+  @Nullable
   @Override
   protected String getRunnableName() {
-    return System.getenv(EnvKeys.TWILL_RUNNABLE_NAME);
+    // No runnable name for the AM
+    return null;
   }
 
   /**
@@ -135,13 +154,13 @@ public final class ApplicationMasterMain extends ServiceMain {
     private static final Logger LOG = LoggerFactory.getLogger(ApplicationKafkaService.class);
 
     private final ZKClient zkClient;
-    private final String kafkaZKPath;
     private final EmbeddedKafkaServer kafkaServer;
+    private final String kafkaZKPath;
 
-    private ApplicationKafkaService(ZKClient zkClient, RunId runId) {
+    private ApplicationKafkaService(ZKClient zkClient, String kafkaZKConnect) {
       this.zkClient = zkClient;
-      this.kafkaZKPath = "/" + runId.getId() + "/kafka";
-      this.kafkaServer = new EmbeddedKafkaServer(generateKafkaConfig(zkClient.getConnectString() + kafkaZKPath));
+      this.kafkaServer = new EmbeddedKafkaServer(generateKafkaConfig(kafkaZKConnect));
+      this.kafkaZKPath = kafkaZKConnect.substring(zkClient.getConnectString().length());
     }
 
     @Override
@@ -236,7 +255,7 @@ public final class ApplicationMasterMain extends ServiceMain {
     private static final Logger LOG = LoggerFactory.getLogger(AppMasterTwillZKPathService.class);
     private final ZKClient zkClient;
 
-    public AppMasterTwillZKPathService(ZKClient zkClient, RunId runId) {
+    AppMasterTwillZKPathService(ZKClient zkClient, RunId runId) {
       super(zkClient, runId);
       this.zkClient = zkClient;
     }
@@ -258,8 +277,7 @@ public final class ApplicationMasterMain extends ServiceMain {
 
       // Try to delete children under /discovery. It may fail with NotEmptyException if there are other instances
       // of the same app running that has discovery services running.
-      List<String> children = zkClient.getChildren(Constants.DISCOVERY_PATH_PREFIX)
-                                      .get(TIMEOUT_SECONDS, TimeUnit.SECONDS).getChildren();
+      List<String> children = getChildren(Constants.DISCOVERY_PATH_PREFIX);
       List<OperationFuture<?>> deleteFutures = new ArrayList<>();
       for (String child : children) {
         String path = Constants.DISCOVERY_PATH_PREFIX + "/" + child;
@@ -272,8 +290,15 @@ public final class ApplicationMasterMain extends ServiceMain {
           future.get();
         } catch (ExecutionException e) {
           if (e.getCause() instanceof KeeperException.NotEmptyException) {
+            // If any deletion of the service failed with not empty, if means there are other apps running,
+            // hence just return
             return;
           }
+          if (e.getCause() instanceof KeeperException.NoNodeException) {
+            // If the service node is gone, it maybe deleted by another app instance that is also shutting down,
+            // hence just keep going
+            continue;
+          }
           throw e;
         }
       }
@@ -304,6 +329,29 @@ public final class ApplicationMasterMain extends ServiceMain {
         if (e.getCause() instanceof KeeperException.NotEmptyException) {
           return false;
         }
+        if (e.getCause() instanceof KeeperException.NoNodeException) {
+          // If the node to be deleted was not created or is already gone, it is the same as delete successfully.
+          return true;
+        }
+        throw e;
+      }
+    }
+
+    /**
+     * Returns the list of children node under the given path.
+     *
+     * @param path path to get children
+     * @return the list of children or empty list if the path doesn't exist.
+     * @throws Exception if failed to get children
+     */
+    private List<String> getChildren(String path) throws Exception {
+      try {
+        return zkClient.getChildren(path).get(TIMEOUT_SECONDS, TimeUnit.SECONDS).getChildren();
+      } catch (ExecutionException e) {
+        if (e.getCause() instanceof KeeperException.NoNodeException) {
+          // If the node doesn't exists, return an empty list
+          return Collections.emptyList();
+        }
         throw e;
       }
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index 368c7b8..0f647cd 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -157,7 +157,8 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
 
     this.amLiveNode = new ApplicationMasterLiveNodeData(Integer.parseInt(System.getenv(EnvKeys.YARN_APP_ID)),
                                                         Long.parseLong(System.getenv(EnvKeys.YARN_APP_ID_CLUSTER_TIME)),
-                                                        amClient.getContainerId().toString(), getLocalizeFiles());
+                                                        amClient.getContainerId().toString(), getLocalizeFiles(),
+                                                        twillRuntimeSpec.getKafkaZKConnect());
 
     this.expectedContainers = new ExpectedContainers(twillSpec);
     this.runningContainers = createRunningContainers(amClient.getContainerId(), amClient.getHost());
@@ -657,8 +658,6 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
       if (environments.containsKey(runnableName)) {
         env.putAll(environments.get(runnableName));
       }
-      // Override with system env
-      env.put(EnvKeys.TWILL_LOG_KAFKA_ZK, getKafkaZKConnect());
 
       ProcessLauncher.PrepareLaunchContext launchContext = processLauncher.prepareLaunch(env,
                                                                                          amLiveNode.getLocalFiles(),
@@ -714,10 +713,6 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
     return String.format("/%s/runnables/%s", runId.getId(), runnableName);
   }
 
-  String getKafkaZKConnect() {
-    return String.format("%s/%s/kafka", zkClient.getConnectString(), runId.getId());
-  }
-
   /**
    * Attempts to change the number of running instances.
    *

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
index a5efb41..2baaca1 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
@@ -66,20 +66,26 @@ public final class TwillContainerMain extends ServiceMain {
 
   private static final Logger LOG = LoggerFactory.getLogger(TwillContainerMain.class);
 
+  private final TwillRuntimeSpecification twillRuntimeSpec;
+
   /**
    * Main method for launching a {@link TwillContainerService} which runs
    * a {@link org.apache.twill.api.TwillRunnable}.
    */
   public static void main(String[] args) throws Exception {
-    new TwillContainerMain().doMain();
+    File twillSpecFile = new File(Constants.Files.RUNTIME_CONFIG_JAR, Constants.Files.TWILL_SPEC);
+    TwillRuntimeSpecification twillRuntimeSpec = TwillRuntimeSpecificationAdapter.create().fromJson(twillSpecFile);
+
+    new TwillContainerMain(twillRuntimeSpec).doMain();
+  }
+
+  private TwillContainerMain(TwillRuntimeSpecification twillRuntimeSpec) {
+    this.twillRuntimeSpec = twillRuntimeSpec;
   }
 
   private void doMain() throws Exception {
     // Try to load the secure store from localized file, which AM requested RM to localize it for this container.
     loadSecureStore();
-    File twillSpecFile = new File(Constants.Files.RUNTIME_CONFIG_JAR, Constants.Files.TWILL_SPEC);
-    TwillRuntimeSpecification twillRuntimeSpec = loadTwillSpec(twillSpecFile);
-    String zkConnectStr = twillRuntimeSpec.getZkConnectStr();
     RunId appRunId = twillRuntimeSpec.getTwillAppRunId();
     RunId runId = RunIds.fromString(System.getenv(EnvKeys.TWILL_RUN_ID));
     String runnableName = System.getenv(EnvKeys.TWILL_RUNNABLE_NAME);
@@ -98,7 +104,7 @@ public final class TwillContainerMain extends ServiceMain {
       logLevels.putAll(dynamicLogLevels);
     }
 
-    ZKClientService zkClientService = createZKClient(zkConnectStr, twillRuntimeSpec.getTwillAppName());
+    ZKClientService zkClientService = createZKClient();
     ZKDiscoveryService discoveryService = new ZKDiscoveryService(zkClientService);
 
     ZKClient appRunZkClient = getAppRunZKClient(zkClientService, appRunId);
@@ -170,12 +176,6 @@ public final class TwillContainerMain extends ServiceMain {
     return classLoader;
   }
 
-  private static TwillRuntimeSpecification loadTwillSpec(File specFile) throws IOException {
-    try (Reader reader = Files.newReader(specFile, Charsets.UTF_8)) {
-      return TwillRuntimeSpecificationAdapter.create().fromJson(reader);
-    }
-  }
-
   private static Map<String, Map<String, String>> loadLogLevels() throws IOException {
     File file = new File(Constants.Files.LOG_LEVELS);
     if (file.exists()) {
@@ -198,8 +198,8 @@ public final class TwillContainerMain extends ServiceMain {
   }
 
   @Override
-  protected String getKafkaZKConnect() {
-    return System.getenv(EnvKeys.TWILL_LOG_KAFKA_ZK);
+  protected TwillRuntimeSpecification getTwillRuntimeSpecification() {
+    return twillRuntimeSpec;
   }
 
   @Override
@@ -207,7 +207,6 @@ public final class TwillContainerMain extends ServiceMain {
     return System.getenv(EnvKeys.TWILL_RUNNABLE_NAME);
   }
 
-
   /**
    * Simple service that force flushing logs on stop.
    */

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
index 61306d6..1945731 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
@@ -77,7 +77,7 @@ final class YarnTwillController extends AbstractTwillController implements Twill
    */
   YarnTwillController(String appName, RunId runId, ZKClient zkClient,
                       final ApplicationMasterLiveNodeData amLiveNodeData, final YarnAppClient yarnAppClient) {
-    super(runId, zkClient, Collections.<LogHandler>emptyList());
+    super(appName, runId, zkClient, amLiveNodeData.getKafkaZKConnect() != null, Collections.<LogHandler>emptyList());
     this.appName = appName;
     this.amLiveNodeData = amLiveNodeData;
     this.startUp = new Callable<ProcessController<YarnApplicationReport>>() {
@@ -91,10 +91,10 @@ final class YarnTwillController extends AbstractTwillController implements Twill
     this.startTimeoutUnit = TimeUnit.SECONDS;
   }
 
-  YarnTwillController(String appName, RunId runId, ZKClient zkClient, Iterable<LogHandler> logHandlers,
-                      Callable<ProcessController<YarnApplicationReport>> startUp,
+  YarnTwillController(String appName, RunId runId, ZKClient zkClient, boolean logCollectionEnabled,
+                      Iterable<LogHandler> logHandlers, Callable<ProcessController<YarnApplicationReport>> startUp,
                       long startTimeout, TimeUnit startTimeoutUnit) {
-    super(runId, zkClient, logHandlers);
+    super(appName, runId, zkClient, logCollectionEnabled, logHandlers);
     this.appName = appName;
     this.startUp = startUp;
     this.startTimeout = startTimeout;

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillControllerFactory.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillControllerFactory.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillControllerFactory.java
index 61fb7cc..40de6a6 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillControllerFactory.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillControllerFactory.java
@@ -30,7 +30,7 @@ import java.util.concurrent.TimeUnit;
  */
 interface YarnTwillControllerFactory {
 
-  YarnTwillController create(RunId runId, Iterable<LogHandler> logHandlers,
+  YarnTwillController create(RunId runId, boolean logCollectionEnabled, Iterable<LogHandler> logHandlers,
                              Callable<ProcessController<YarnApplicationReport>> startUp,
                              long startTimeout, TimeUnit startTimeoutUnit);
 }

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index 2d1edd0..4846fe3 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -419,7 +419,10 @@ final class YarnTwillPreparer implements TwillPreparer {
           }
         };
 
-      YarnTwillController controller = controllerFactory.create(runId, logHandlers, submitTask, timeout, timeoutUnit);
+      boolean logCollectionEnabled = config.getBoolean(Configs.Keys.LOG_COLLECTION_ENABLED,
+                                                       Configs.Defaults.LOG_COLLECTION_ENABLED);
+      YarnTwillController controller = controllerFactory.create(runId, logCollectionEnabled,
+                                                                logHandlers, submitTask, timeout, timeoutUnit);
       controller.start();
       return controller;
     } catch (Exception e) {
@@ -671,11 +674,13 @@ final class YarnTwillPreparer implements TwillPreparer {
       }
       TwillSpecification newTwillSpec = new DefaultTwillSpecification(spec.getName(), runtimeSpec, spec.getOrders(),
                                                                       spec.getPlacementPolicies(), eventHandler);
+      boolean logCollectionEnabled = config.getBoolean(Configs.Keys.LOG_COLLECTION_ENABLED,
+                                                       Configs.Defaults.LOG_COLLECTION_ENABLED);
       TwillRuntimeSpecificationAdapter.create().toJson(
         new TwillRuntimeSpecification(newTwillSpec, appLocation.getLocationFactory().getHomeLocation().getName(),
                                       appLocation.toURI(), zkConnectString, runId, twillSpec.getName(),
                                       getReservedMemory(), config.get(YarnConfiguration.RM_SCHEDULER_ADDRESS),
-                                      logLevels, maxRetries, getMinHeapRatio()), writer);
+                                      logLevels, maxRetries, getMinHeapRatio(), logCollectionEnabled), writer);
     }
     LOG.debug("Done {}", targetFile);
   }

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index d8e48de..d2b53b9 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -291,11 +291,12 @@ public final class YarnTwillRunnerService implements TwillRunnerService {
                                  appLocation, twillClassPaths, jvmOptions,
                                  locationCache, new YarnTwillControllerFactory() {
       @Override
-      public YarnTwillController create(RunId runId, Iterable<LogHandler> logHandlers,
+      public YarnTwillController create(RunId runId, boolean logCollectionEnabled, Iterable<LogHandler> logHandlers,
                                         Callable<ProcessController<YarnApplicationReport>> startUp,
                                         long startTimeout, TimeUnit startTimeoutUnit) {
         ZKClient zkClient = ZKClients.namespace(zkClientService, "/" + appName);
         YarnTwillController controller = listenController(new YarnTwillController(appName, runId, zkClient,
+                                                                                  logCollectionEnabled,
                                                                                   logHandlers, startUp,
                                                                                   startTimeout, startTimeoutUnit));
         synchronized (YarnTwillRunnerService.this) {

http://git-wip-us.apache.org/repos/asf/twill/blob/3045b91b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
index ad0a837..902c146 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
@@ -17,26 +17,32 @@
  */
 package org.apache.twill.yarn;
 
+import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.Configs;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
 import org.apache.twill.api.logging.LogEntry;
 import org.apache.twill.api.logging.LogHandler;
 import org.apache.twill.api.logging.LogThrowable;
 import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.common.Threads;
+import org.apache.twill.discovery.ServiceDiscovered;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.PrintWriter;
+import java.util.Collections;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * Test for LogHandler able to receive logs from AM and runnable.
@@ -100,6 +106,42 @@ public class LogHandlerTestRun extends BaseYarnTest {
     Assert.assertEquals("Exception", t.getMessage());
   }
 
+  @Test
+  public void testDisableLogCollection() throws Exception {
+    final AtomicBoolean logReceived = new AtomicBoolean();
+
+    // Start the LogRunnable by turning off log collection
+    TwillRunner runner = getTwillRunner();
+    TwillController controller = runner.prepare(new LogRunnable())
+      .withConfiguration(Collections.singletonMap(Configs.Keys.LOG_COLLECTION_ENABLED, "false"))
+      .addLogHandler(new LogHandler() {
+        @Override
+        public void onLog(LogEntry logEntry) {
+          logReceived.set(true);
+        }
+      })
+      .start();
+
+    // Make sure the runnable gets executed
+    try {
+      final CountDownLatch latch = new CountDownLatch(1);
+      controller.discoverService("log").watchChanges(new ServiceDiscovered.ChangeListener() {
+        @Override
+        public void onChange(ServiceDiscovered serviceDiscovered) {
+          if (Iterables.size(serviceDiscovered) == 1) {
+            latch.countDown();
+          }
+        }
+      }, Threads.SAME_THREAD_EXECUTOR);
+      Assert.assertTrue(latch.await(120, TimeUnit.SECONDS));
+    } finally {
+      controller.terminate().get(120, TimeUnit.SECONDS);
+    }
+
+    // Should receive no log
+    Assert.assertFalse("Not expecting logs collected", logReceived.get());
+  }
+
   /**
    * TwillRunnable for the test case to simply emit one log line.
    */
@@ -121,6 +163,8 @@ public class LogHandlerTestRun extends BaseYarnTest {
       } catch (Throwable t) {
         LOG.error("Got exception", t);
       }
+      // Announce so that test case knows the code reaches here.
+      getContext().announce("log", 12345);
 
       Uninterruptibles.awaitUninterruptibly(stopLatch);
     }


[09/24] twill git commit: (TWILL-171) Clone the HDFS delegation in HA mode.

Posted by ch...@apache.org.
(TWILL-171) Clone the HDFS delegation in HA mode.

- This is for working around HDFS-9276

This closes #42 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: d2a503ac7032fb0874f414acad16015854e54e56
Parents: d1f1122
Author: Terence Yim <ch...@apache.org>
Authored: Mon Mar 27 10:51:26 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Mar 27 12:37:55 2017 -0700

----------------------------------------------------------------------
 .../appmaster/ApplicationMasterMain.java        |  2 +-
 .../appmaster/ApplicationMasterService.java     |  8 +++--
 .../internal/container/TwillContainerMain.java  |  2 +-
 .../container/TwillContainerService.java        |  7 +++--
 .../internal/yarn/AbstractYarnTwillService.java | 18 +++++++++--
 .../apache/twill/internal/yarn/YarnUtils.java   | 33 ++++++++++++++++++++
 6 files changed, 60 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/d2a503ac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
index 81c61ac..445656d 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
@@ -84,7 +84,7 @@ public final class ApplicationMasterMain extends ServiceMain {
 
     final YarnAMClient amClient = new VersionDetectYarnAMClientFactory(conf).create();
     ApplicationMasterService service =
-      new ApplicationMasterService(runId, zkClientService, twillRuntimeSpec, amClient,
+      new ApplicationMasterService(runId, zkClientService, twillRuntimeSpec, amClient, conf,
                                    createAppLocation(conf, twillRuntimeSpec.getFsUser(),
                                                      twillRuntimeSpec.getTwillAppDir()));
     TrackerService trackerService = new TrackerService(service);

http://git-wip-us.apache.org/repos/asf/twill/blob/d2a503ac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index 0f647cd..523ffce 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -37,6 +37,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -138,9 +139,10 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
   private Queue<RunnableContainerRequest> runnableContainerRequests;
   private ExecutorService instanceChangeExecutor;
 
-  public ApplicationMasterService(RunId runId, ZKClient zkClient, TwillRuntimeSpecification twillRuntimeSpec,
-                                  YarnAMClient amClient, Location applicationLocation) throws Exception {
-    super(zkClient, runId, applicationLocation);
+  public ApplicationMasterService(RunId runId, ZKClient zkClient,
+                                  TwillRuntimeSpecification twillRuntimeSpec, YarnAMClient amClient,
+                                  Configuration config, Location applicationLocation) throws Exception {
+    super(zkClient, runId, config, applicationLocation);
 
     this.runId = runId;
     this.twillRuntimeSpec = twillRuntimeSpec;

http://git-wip-us.apache.org/repos/asf/twill/blob/d2a503ac/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
index 2baaca1..e6d86a5 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
@@ -124,7 +124,7 @@ public final class TwillContainerMain extends ServiceMain {
     ZKClient containerZKClient = getContainerZKClient(zkClientService, appRunId, runnableName);
     Configuration conf = new YarnConfiguration(new HdfsConfiguration(new Configuration()));
     TwillContainerService service = new TwillContainerService(context, containerInfo, containerZKClient,
-                                                              runId, runnableSpec, getClassLoader(),
+                                                              runId, runnableSpec, getClassLoader(), conf,
                                                               createAppLocation(conf, twillRuntimeSpec.getFsUser(),
                                                                                 twillRuntimeSpec.getTwillAppDir()),
                                                               defaultLogLevels, logLevels);

http://git-wip-us.apache.org/repos/asf/twill/blob/d2a503ac/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
index 58298a0..6335f9f 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
@@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.twill.api.Command;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.TwillRunnable;
@@ -70,9 +71,9 @@ public final class TwillContainerService extends AbstractYarnTwillService {
 
   TwillContainerService(BasicTwillContext context, ContainerInfo containerInfo, ZKClient zkClient,
                         RunId runId, TwillRunnableSpecification specification, ClassLoader classLoader,
-                        Location applicationLocation, Map<String, String> defaultLogLevels,
-                        Map<String, String> logLevels) {
-    super(zkClient, runId, applicationLocation);
+                        Configuration config, Location applicationLocation,
+                        Map<String, String> defaultLogLevels, Map<String, String> logLevels) {
+    super(zkClient, runId, config, applicationLocation);
 
     this.specification = specification;
     this.classLoader = classLoader;

http://git-wip-us.apache.org/repos/asf/twill/blob/d2a503ac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnTwillService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnTwillService.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnTwillService.java
index 64f81b4..44cabdc 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnTwillService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnTwillService.java
@@ -17,6 +17,7 @@
  */
 package org.apache.twill.internal.yarn;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.twill.api.RunId;
@@ -40,11 +41,15 @@ import java.io.IOException;
 public abstract class AbstractYarnTwillService extends AbstractTwillService {
 
   private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnTwillService.class);
-  protected final Location applicationLocation;
+
+  private final Configuration config;
+  private final Location applicationLocation;
   protected volatile Credentials credentials;
 
-  protected AbstractYarnTwillService(ZKClient zkClient, RunId runId, Location applicationLocation) {
+  protected AbstractYarnTwillService(ZKClient zkClient, RunId runId,
+                                     Configuration config, Location applicationLocation) {
     super(zkClient, runId);
+    this.config = config;
     this.applicationLocation = applicationLocation;
   }
 
@@ -83,11 +88,20 @@ public abstract class AbstractYarnTwillService extends AbstractTwillService {
     try {
       Credentials credentials = new Credentials();
       Location location = getSecureStoreLocation();
+
+      // If failed to determine the secure store location, simply ignore the message.
+      if (location == null) {
+        return true;
+      }
+
       try (DataInputStream input = new DataInputStream(new BufferedInputStream(location.getInputStream()))) {
         credentials.readTokenStorageStream(input);
       }
 
       UserGroupInformation.getCurrentUser().addCredentials(credentials);
+
+      // Clone the HDFS tokens for HA NameNode. This is to workaround bug HDFS-9276.
+      YarnUtils.cloneHaNnCredentials(config);
       this.credentials = credentials;
 
       LOG.info("Secure store updated from {}.", location);

http://git-wip-us.apache.org/repos/asf/twill/blob/d2a503ac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
index 9574554..ff8f4bb 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
@@ -21,9 +21,12 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.io.DataInputByteBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.Credentials;
@@ -48,8 +51,10 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
@@ -178,6 +183,34 @@ public class YarnUtils {
   }
 
   /**
+   * Clones the delegation token to individual host behind the same logical address.
+   *
+   * @param config the hadoop configuration
+   * @throws IOException if failed to get information for the current user.
+   */
+  public static void cloneHaNnCredentials(Configuration config) throws IOException {
+    String scheme = URI.create(config.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+                                          CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT)).getScheme();
+
+    // Loop through all name services. Each name service could have multiple name node associated with it.
+    for (Map.Entry<String, Map<String, InetSocketAddress>> entry : DFSUtil.getHaNnRpcAddresses(config).entrySet()) {
+      String nsId = entry.getKey();
+      Map<String, InetSocketAddress> addressesInNN = entry.getValue();
+      if (!HAUtil.isHAEnabled(config, nsId) || addressesInNN == null || addressesInNN.isEmpty()) {
+        continue;
+      }
+
+      // The client may have a delegation token set for the logical
+      // URI of the cluster. Clone this token to apply to each of the
+      // underlying IPC addresses so that the IPC code can find it.
+      URI uri = URI.create(scheme + "://" + nsId);
+
+      LOG.info("Cloning delegation token for uri {}", uri);
+      HAUtil.cloneDelegationTokenForLogicalUri(UserGroupInformation.getCurrentUser(), uri, addressesInNN.values());
+    }
+  }
+
+  /**
    * Encodes the given {@link Credentials} as bytes.
    */
   public static ByteBuffer encodeCredentials(Credentials credentials) {


[14/24] twill git commit: (TWILL-229) Default to use logback_template.xml to configure logger, unless overridden by user

Posted by ch...@apache.org.
(TWILL-229) Default to use logback_template.xml to configure logger, unless overridden by user

This closes #49 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/82e59422
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/82e59422
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/82e59422

Branch: refs/heads/site
Commit: 82e5942226f4608191bef0b936bfacc0e34e33fe
Parents: 2c42097
Author: Terence Yim <ch...@apache.org>
Authored: Thu Mar 30 14:23:36 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Thu Mar 30 15:44:05 2017 -0700

----------------------------------------------------------------------
 .../org/apache/twill/internal/ServiceMain.java   | 19 ++++++++++++++++---
 1 file changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/82e59422/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
index 0bf07e0..b27dcd8 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
@@ -18,6 +18,8 @@
 package org.apache.twill.internal;
 
 import ch.qos.logback.classic.LoggerContext;
+import ch.qos.logback.classic.util.ContextInitializer;
+import ch.qos.logback.core.joran.spi.JoranException;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.common.util.concurrent.Futures;
@@ -44,9 +46,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
+import java.net.MalformedURLException;
 import java.net.URI;
+import java.net.URL;
 import java.security.PrivilegedExceptionAction;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
 
@@ -65,7 +68,7 @@ public abstract class ServiceMain {
   }
 
   protected final void doMain(final Service mainService,
-                              Service...prerequisites) throws ExecutionException, InterruptedException {
+                              Service...prerequisites) throws Exception {
     // Only configure the log collection if it is enabled.
     if (getTwillRuntimeSpecification().isLogCollectionEnabled()) {
       configureLogger();
@@ -189,7 +192,7 @@ public abstract class ServiceMain {
     );
   }
 
-  private void configureLogger() {
+  private void configureLogger() throws MalformedURLException, JoranException {
     // Check if SLF4J is bound to logback in the current environment
     ILoggerFactory loggerFactory = LoggerFactory.getILoggerFactory();
     if (!(loggerFactory instanceof LoggerContext)) {
@@ -198,6 +201,16 @@ public abstract class ServiceMain {
 
     LoggerContext context = (LoggerContext) loggerFactory;
 
+    ContextInitializer contextInitializer = new ContextInitializer(context);
+    URL url = contextInitializer.findURLOfDefaultConfigurationFile(false);
+    if (url == null) {
+      // The logger context was not initialized using configuration file, initialize it with the logback template.
+      File twillLogback = new File(Constants.Files.RUNTIME_CONFIG_JAR, Constants.Files.LOGBACK_TEMPLATE);
+      if (twillLogback.exists()) {
+        contextInitializer.configureByResource(twillLogback.toURI().toURL());
+      }
+    }
+
     // Attach the KafkaAppender to the root logger
     KafkaAppender kafkaAppender = new KafkaAppender();
     kafkaAppender.setName("KAFKA");


[05/24] twill git commit: (Twill-222) fix for default root log level

Posted by ch...@apache.org.
(Twill-222) fix for default root log level

This closes #37 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: c4cceef5e797b1d15fe36b09f004c579df10725e
Parents: a3e4d38
Author: yaojiefeng <ya...@cask.co>
Authored: Thu Mar 16 16:57:05 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Sat Mar 18 01:31:59 2017 -0700

----------------------------------------------------------------------
 .../appmaster/ApplicationMasterService.java     |  3 ---
 .../internal/container/TwillContainerMain.java  |  7 +++++-
 .../apache/twill/yarn/YarnTwillPreparer.java    |  3 ---
 .../org/apache/twill/yarn/LogLevelTestRun.java  | 26 +++++++++++---------
 4 files changed, 21 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/c4cceef5/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index b4ac288..368c7b8 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -650,9 +650,6 @@ public final class ApplicationMasterService extends AbstractYarnTwillService imp
       String runnableName = provisionRequest.getRuntimeSpec().getName();
       LOG.info("Starting runnable {} in {}", runnableName, processLauncher.getContainerInfo().getContainer());
 
-      LOG.debug("Log level for Twill runnable {} is {}", runnableName,
-                twillRuntimeSpec.getLogLevels().get(runnableName).get(Logger.ROOT_LOGGER_NAME));
-
       int containerCount = expectedContainers.getExpected(runnableName);
 
       // Setup container environment variables

http://git-wip-us.apache.org/repos/asf/twill/blob/c4cceef5/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
index e37cd44..a5efb41 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
@@ -18,6 +18,7 @@
 package org.apache.twill.internal.container;
 
 import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.io.Files;
 import com.google.common.reflect.TypeToken;
 import com.google.common.util.concurrent.AbstractIdleService;
@@ -88,7 +89,11 @@ public final class TwillContainerMain extends ServiceMain {
     Map<String, String> dynamicLogLevels = loadLogLevels().get(runnableName);
 
     Map<String, String> logLevels = new HashMap<>();
-    logLevels.putAll(defaultLogLevels);
+    if (defaultLogLevels != null) {
+      logLevels.putAll(defaultLogLevels);
+    } else {
+      defaultLogLevels = ImmutableMap.of();
+    }
     if (dynamicLogLevels != null) {
       logLevels.putAll(dynamicLogLevels);
     }

http://git-wip-us.apache.org/repos/asf/twill/blob/c4cceef5/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index d9e70fd..de03a7a 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -178,9 +178,6 @@ final class YarnTwillPreparer implements TwillPreparer {
     this.classAcceptor = new ClassAcceptor();
     this.locationCache = locationCache;
     this.twillClassPaths = twillClassPaths;
-
-    // By default, the root logger is having INFO log level
-    setLogLevel(LogEntry.Level.INFO);
   }
 
   private void confirmRunnableName(String runnableName) {

http://git-wip-us.apache.org/repos/asf/twill/blob/c4cceef5/twill-yarn/src/test/java/org/apache/twill/yarn/LogLevelTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LogLevelTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LogLevelTestRun.java
index 771797d..717a80f 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/LogLevelTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LogLevelTestRun.java
@@ -39,6 +39,7 @@ import java.io.PrintWriter;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 
 /**
  * Test class whether enable certain log level for application container works.
@@ -111,11 +112,17 @@ public class LogLevelTestRun extends BaseYarnTest {
     testLogLevel("ALL");
   }
 
+  @Test
+  public void testNoSetLogLevel() throws Exception {
+    testLogLevel("NONE");
+  }
+
   private void testLogLevel(String method) throws Exception {
     YarnTwillRunnerService runner = getTwillRunner();
     runner.start();
 
     TwillPreparer preparer = runner.prepare(new LogLevelTestApplication());
+    // Set log level to DEBUG
     if (method.equals("ROOT")) {
       preparer.setLogLevel(LogEntry.Level.DEBUG);
     }
@@ -125,7 +132,6 @@ public class LogLevelTestRun extends BaseYarnTest {
     if (method.equals("RUNNABLE")) {
       preparer.setLogLevels(LogLevelTestRunnable.class.getSimpleName(), defaultLogArguments);
     }
-    // Set log level to DEBUG
     TwillController controller = preparer
       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out)))
       .start();
@@ -140,11 +146,9 @@ public class LogLevelTestRun extends BaseYarnTest {
     }, Threads.SAME_THREAD_EXECUTOR);
     Assert.assertTrue(running.await(200, TimeUnit.SECONDS));
 
-    LogEntry.Level logLevel = waitForLogLevel(controller, LogLevelTestRunnable.class.getSimpleName(), 30L,
-                                              TimeUnit.SECONDS);
-
-    // Verify we got DEBUG log level.
-    Assert.assertEquals(LogEntry.Level.DEBUG, logLevel);
+    // If we do not set the root log level, it should be null from resource report.
+    Assert.assertTrue(waitForLogLevel(controller, LogLevelTestRunnable.class.getSimpleName(), 30L,
+                                      TimeUnit.SECONDS, !method.equals("NONE") ? LogEntry.Level.DEBUG : null));
 
     controller.terminate().get(120, TimeUnit.SECONDS);
 
@@ -154,8 +158,8 @@ public class LogLevelTestRun extends BaseYarnTest {
 
   // Need helper method here to wait for getting resource report because {@link TwillController#getResourceReport()}
   // could return null if the application has not fully started.
-  private LogEntry.Level waitForLogLevel(TwillController controller, String runnable, long timeout,
-                                         TimeUnit timeoutUnit) throws InterruptedException {
+  private boolean waitForLogLevel(TwillController controller, String runnable, long timeout,
+                                  TimeUnit timeoutUnit, @Nullable LogEntry.Level expected) throws InterruptedException {
 
     Stopwatch stopwatch = new Stopwatch();
     stopwatch.start();
@@ -166,13 +170,13 @@ public class LogLevelTestRun extends BaseYarnTest {
       }
       for (TwillRunResources resources : report.getRunnableResources(runnable)) {
         LogEntry.Level level = resources.getLogLevels().get(Logger.ROOT_LOGGER_NAME);
-        if (level != null) {
-           return level;
+        if (expected == level) {
+           return true;
         }
       }
       TimeUnit.MILLISECONDS.sleep(100);
     } while (stopwatch.elapsedTime(timeoutUnit) < timeout);
 
-    return null;
+    return false;
   }
 }


[08/24] twill git commit: Add website link to readme

Posted by ch...@apache.org.
Add website link to readme

This closes #41 on Github.

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: d1f1122b9f939d768d97c1fc55269ae65934ddd0
Parents: 3045b91
Author: Keith Turner <kt...@apache.org>
Authored: Mon Mar 27 11:48:08 2017 -0400
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Mar 27 11:42:14 2017 -0700

----------------------------------------------------------------------
 README.md | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/d1f1122b/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index ae0daf8..e042365 100644
--- a/README.md
+++ b/README.md
@@ -18,7 +18,7 @@
 
 What is Apache Twill?
 ---------------------
-Twill is an abstraction over Apache Hadoop� YARN that reduces the complexity 
+[Twill] is an abstraction over Apache Hadoop� YARN that reduces the complexity 
 of developing distributed applications, allowing developers to focus more on 
 their business logic. Twill allows you to use YARN\u2019s distributed capabilities 
 with a programming model that is similar to running threads.
@@ -62,3 +62,5 @@ Apache Twill uses the built-in java cryptography libraries for unique ID
 generation. See 
 <http://www.oracle.com/us/products/export/export-regulations-345813.html>
 for more details on Java's cryptography features.
+
+[Twill]: http://twill.apache.org/


[22/24] twill git commit: Bump version to 0.12.0-SNAPSHOT

Posted by ch...@apache.org.
Bump version to 0.12.0-SNAPSHOT


Project: http://git-wip-us.apache.org/repos/asf/twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/twill/commit/73139adf
Tree: http://git-wip-us.apache.org/repos/asf/twill/tree/73139adf
Diff: http://git-wip-us.apache.org/repos/asf/twill/diff/73139adf

Branch: refs/heads/site
Commit: 73139adf17f9c40f5761bbdb00d01d9f5ca9d9c2
Parents: e8fd829
Author: Terence Yim <ch...@apache.org>
Authored: Tue Apr 4 02:47:14 2017 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Tue Apr 4 02:47:14 2017 -0700

----------------------------------------------------------------------
 pom.xml                      | 2 +-
 twill-api/pom.xml            | 2 +-
 twill-common/pom.xml         | 2 +-
 twill-core/pom.xml           | 2 +-
 twill-discovery-api/pom.xml  | 2 +-
 twill-discovery-core/pom.xml | 2 +-
 twill-examples/echo/pom.xml  | 2 +-
 twill-examples/pom.xml       | 2 +-
 twill-examples/yarn/pom.xml  | 2 +-
 twill-ext/pom.xml            | 2 +-
 twill-java8-test/pom.xml     | 2 +-
 twill-yarn/pom.xml           | 2 +-
 twill-zookeeper/pom.xml      | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 680d9a7..3fbbcdf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.twill</groupId>
     <artifactId>twill-parent</artifactId>
-    <version>0.11.0</version>
+    <version>0.12.0-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Apache Twill</name>
     <url>http://twill.apache.org</url>

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index 14afaee..6f6b67d 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-api</artifactId>

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index 000871c..8baccaf 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index 90dc20f..7e24154 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index eae740f..7626503 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index 6d5ee06..c710c49 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-examples/echo/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/echo/pom.xml b/twill-examples/echo/pom.xml
index 63ac83b..43a7280 100644
--- a/twill-examples/echo/pom.xml
+++ b/twill-examples/echo/pom.xml
@@ -25,7 +25,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-examples</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
 
     <name>Apache Twill examples: Echo</name>

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-examples/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/pom.xml b/twill-examples/pom.xml
index 84e65c6..936ab18 100644
--- a/twill-examples/pom.xml
+++ b/twill-examples/pom.xml
@@ -25,7 +25,7 @@ limitations under the License.
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-examples/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/yarn/pom.xml b/twill-examples/yarn/pom.xml
index 2a8243c..6530ee2 100644
--- a/twill-examples/yarn/pom.xml
+++ b/twill-examples/yarn/pom.xml
@@ -24,7 +24,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-examples</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
 
     <name>Apache Twill examples: YARN</name>

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-ext/pom.xml
----------------------------------------------------------------------
diff --git a/twill-ext/pom.xml b/twill-ext/pom.xml
index 18abe8a..722a8fc 100644
--- a/twill-ext/pom.xml
+++ b/twill-ext/pom.xml
@@ -22,7 +22,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-java8-test/pom.xml
----------------------------------------------------------------------
diff --git a/twill-java8-test/pom.xml b/twill-java8-test/pom.xml
index 7f21aad..aec8656 100644
--- a/twill-java8-test/pom.xml
+++ b/twill-java8-test/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-java8-test</artifactId>

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index 786d295..8161e90 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/twill/blob/73139adf/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index ed6df67..3b0456d 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.11.0</version>
+        <version>0.12.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>