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 2014/04/22 08:06:39 UTC

[42/50] [abbrv] git commit: (TWILL-71) move debug options from runner to preparer

(TWILL-71) move debug options from runner to preparer


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

Branch: refs/heads/site
Commit: 62e51ebb745220cb74c962927adca182ec24def3
Parents: 9da2d3f
Author: anew <an...@continuuity.com>
Authored: Fri Apr 11 23:41:42 2014 -0700
Committer: anew <an...@continuuity.com>
Committed: Fri Apr 11 23:41:42 2014 -0700

----------------------------------------------------------------------
 .../org/apache/twill/api/TwillPreparer.java     | 39 +++++++++++++++++
 .../apache/twill/yarn/YarnTwillPreparer.java    | 46 +++++++++++++++-----
 .../twill/yarn/YarnTwillRunnerService.java      | 38 +---------------
 .../org/apache/twill/yarn/DebugTestRun.java     |  2 +-
 .../org/apache/twill/yarn/LocalFileTestRun.java | 11 +----
 5 files changed, 77 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/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 b2a3ce2..6caad70 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
@@ -45,6 +45,45 @@ public interface TwillPreparer {
   TwillPreparer setUser(String user);
 
   /**
+   * This methods sets the extra JVM options that will be passed to the java command line for every runnable
+   * of the application started through this {@link org.apache.twill.api.TwillPreparer} instance.
+   *
+   * This is intended for advance usage. All options will be passed unchanged to the java command line. Invalid
+   * options could cause application not able to start.
+   *
+   * @param options extra JVM options.
+   */
+  TwillPreparer setJVMOptions(String options);
+
+  /**
+   * This methods adds extra JVM options that will be passed to the java command line for every runnable
+   * of the application started through this {@link org.apache.twill.api.TwillPreparer} instance.
+   *
+   * This is intended for advance usage. All options will be passed unchanged to the java command line. Invalid
+   * options could cause application not able to start.
+   *
+   * @param options extra JVM options.
+   */
+  TwillPreparer addJVMOptions(String options);
+
+  /**
+   * Enable debugging for runnables, without suspending the virtual machine to wait for the debugger.
+   * This replaces any previous debug settings.
+   * @param runnables the names of runnables to enable for debugging. If empty, it means all runnables.
+   */
+  TwillPreparer enableDebugging(String ... runnables);
+
+  /**
+   * Enable debugging for runnables. This replaces any previous debug settings.
+   * @param doSuspend whether the virtual machines should be supended until the debugger connects. This
+   *                  option allows to debug a container from the very beginning. Note that in that case,
+   *                  the container cannot notify the controller of its debug port until the debugger is
+   *                  attached - you must figure out where it is running using the YARN console or APIs.
+   * @param runnables the names of runnables to enable for debugging. If empty, it means all runnables.
+   */
+  TwillPreparer enableDebugging(boolean doSuspend, String ... runnables);
+
+  /**
    * Sets the list of arguments that will be passed to the application. The arguments can be retrieved
    * from {@link TwillContext#getApplicationArguments()}.
    *

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/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 a02efea..2ed2de0 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
@@ -21,7 +21,6 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
-import com.google.common.base.Supplier;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.HashMultimap;
@@ -113,7 +112,6 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final YarnAppClient yarnAppClient;
   private final ZKClient zkClient;
   private final LocationFactory locationFactory;
-  private final JvmOptions jvmOpts;
   private final YarnTwillControllerFactory controllerFactory;
   private final RunId runId;
 
@@ -126,22 +124,25 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final Credentials credentials;
   private final int reservedMemory;
   private String user;
+  private String extraOptions;
+  private JvmOptions.DebugOptions debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
 
-  YarnTwillPreparer(YarnConfiguration yarnConfig, TwillSpecification twillSpec, YarnAppClient yarnAppClient,
-                    ZKClient zkClient, LocationFactory locationFactory, Supplier<JvmOptions> jvmOpts,
+  YarnTwillPreparer(YarnConfiguration yarnConfig, TwillSpecification twillSpec,
+                    YarnAppClient yarnAppClient, ZKClient zkClient,
+                    LocationFactory locationFactory, String extraOptions,
                     YarnTwillControllerFactory controllerFactory) {
     this.yarnConfig = yarnConfig;
     this.twillSpec = twillSpec;
     this.yarnAppClient = yarnAppClient;
     this.zkClient = ZKClients.namespace(zkClient, "/" + twillSpec.getName());
     this.locationFactory = locationFactory;
-    this.jvmOpts = jvmOpts.get();
     this.controllerFactory = controllerFactory;
     this.runId = RunIds.generate();
     this.credentials = createCredentials();
     this.reservedMemory = yarnConfig.getInt(Configs.Keys.JAVA_RESERVED_MEMORY_MB,
                                             Configs.Defaults.JAVA_RESERVED_MEMORY_MB);
     this.user = System.getProperty("user.name");
+    this.extraOptions = extraOptions;
   }
 
   @Override
@@ -157,6 +158,29 @@ final class YarnTwillPreparer implements TwillPreparer {
   }
 
   @Override
+  public TwillPreparer setJVMOptions(String options) {
+    this.extraOptions = options;
+    return this;
+  }
+
+  @Override
+  public TwillPreparer addJVMOptions(String options) {
+    this.extraOptions = extraOptions == null ? options : extraOptions + " " + options;
+    return this;
+  }
+
+  @Override
+  public TwillPreparer enableDebugging(String... runnables) {
+    return enableDebugging(false, runnables);
+  }
+
+  @Override
+  public TwillPreparer enableDebugging(boolean doSuspend, String... runnables) {
+    this.debugOptions = new JvmOptions.DebugOptions(true, doSuspend, ImmutableSet.copyOf(runnables));
+    return this;
+  }
+
+  @Override
   public TwillPreparer withApplicationArguments(String... args) {
     return withApplicationArguments(ImmutableList.copyOf(args));
   }
@@ -242,7 +266,7 @@ final class YarnTwillPreparer implements TwillPreparer {
           saveSpecification(twillSpec, runnableLocalFiles, localFiles);
           saveLogback(localFiles);
           saveLauncher(localFiles);
-          saveJvmOptions(jvmOpts, localFiles);
+          saveJvmOptions(localFiles);
           saveArguments(new Arguments(arguments, runnableArgs), localFiles);
           saveLocalFiles(localFiles, ImmutableSet.of(Constants.Files.TWILL_SPEC,
                                                      Constants.Files.LOGBACK_TEMPLATE,
@@ -275,7 +299,7 @@ final class YarnTwillPreparer implements TwillPreparer {
               "-Dtwill.app=$" + EnvKeys.TWILL_APP_NAME,
               "-cp", Constants.Files.LAUNCHER_JAR + ":$HADOOP_CONF_DIR",
               "-Xmx" + (Constants.APP_MASTER_MEMORY_MB - Constants.APP_MASTER_RESERVED_MEMORY_MB) + "m",
-              jvmOpts.getExtraOptions() == null ? "" : jvmOpts.getExtraOptions(),
+              extraOptions == null ? "" : extraOptions,
               TwillLauncher.class.getName(),
               Constants.Files.APP_MASTER_JAR,
               ApplicationMasterMain.class.getName(),
@@ -499,15 +523,15 @@ final class YarnTwillPreparer implements TwillPreparer {
     localFiles.put(Constants.Files.LAUNCHER_JAR, createLocalFile(Constants.Files.LAUNCHER_JAR, location));
   }
 
-  private void saveJvmOptions(JvmOptions opts, Map<String, LocalFile> localFiles) throws IOException {
-    if ((opts.getExtraOptions() == null || opts.getExtraOptions().isEmpty()) &&
-      JvmOptions.DebugOptions.NO_DEBUG.equals(opts.getDebugOptions())) {
+  private void saveJvmOptions(Map<String, LocalFile> localFiles) throws IOException {
+    if ((extraOptions == null || extraOptions.isEmpty()) &&
+      JvmOptions.DebugOptions.NO_DEBUG.equals(debugOptions)) {
       // If no vm options, no need to localize the file.
       return;
     }
     LOG.debug("Create and copy {}", Constants.Files.JVM_OPTIONS);
     final Location location = createTempLocation(Constants.Files.JVM_OPTIONS);
-    JvmOptionsCodec.encode(opts, new OutputSupplier<Writer>() {
+    JvmOptionsCodec.encode(new JvmOptions(extraOptions, debugOptions), new OutputSupplier<Writer>() {
       @Override
       public Writer getOutput() throws IOException {
         return new OutputStreamWriter(location.getOutputStream(), Charsets.UTF_8);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/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 4ecfb0e..dadedce 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
@@ -21,7 +21,6 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
-import com.google.common.base.Suppliers;
 import com.google.common.base.Throwables;
 import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.HashMultimap;
@@ -65,7 +64,6 @@ import org.apache.twill.filesystem.HDFSLocationFactory;
 import org.apache.twill.filesystem.Location;
 import org.apache.twill.filesystem.LocationFactory;
 import org.apache.twill.internal.Constants;
-import org.apache.twill.internal.JvmOptions;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.RunIds;
 import org.apache.twill.internal.SingleRunnableApplication;
@@ -135,7 +133,6 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
   private Cancellable watchCancellable;
 
   private volatile String jvmOptions = null;
-  private volatile JvmOptions.DebugOptions debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
 
   public YarnTwillRunnerService(YarnConfiguration config, String zkConnect) {
     this(config, zkConnect, new HDFSLocationFactory(getFileSystem(config), "/twill"));
@@ -172,38 +169,6 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
     return jvmOptions;
   }
 
-  /**
-   * Disable debugging for runnables. It only affects applications that are started after this
-   * method is called. Note that debugging is disabled by default, and this method is only needed
-   * to revert a previous call of @link #enableDebugging.
-   */
-  public void disableDebugging() {
-    this.debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
-  }
-
-  /**
-   * Enable debugging for runnables, without suspending the virtual machine to wait for the debugger.
-   * This replaces any previous debug settings and only affects applications that are started after
-   * this method is called.
-   * @param runnables the names of runnables to enable for debugging.
-   */
-  public void enableDebugging(String ... runnables) {
-    this.debugOptions = new JvmOptions.DebugOptions(true, false, ImmutableSet.copyOf(runnables));
-  }
-
-  /**
-   * Enable debugging for runnables. This replaces any previous debug settings and only affects
-   * applications that are started after this method is called.
-   * @param doSuspend whether the virtual machines should be supended until the debugger connects. This
-   *                  option allows to debug a container from the very beginning. Note that in that case,
-   *                  the container cannot notify the controller of its debug port until the debugger is
-   *                  attached - you must figure out where it is running using the YARN console or APIs.
-   * @param runnables the names of runnables to enable for debugging.
-   */
-  public void enableDebugging(boolean doSuspend, String ... runnables) {
-    this.debugOptions = new JvmOptions.DebugOptions(true, false, ImmutableSet.copyOf(runnables));
-  }
-
   @Override
   public Cancellable scheduleSecureStoreUpdate(final SecureStoreUpdater updater,
                                                long initialDelay, long delay, TimeUnit unit) {
@@ -273,8 +238,7 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
     final TwillSpecification twillSpec = application.configure();
     final String appName = twillSpec.getName();
 
-    return new YarnTwillPreparer(yarnConfig, twillSpec, yarnAppClient, zkClientService, locationFactory,
-                                 Suppliers.ofInstance(new JvmOptions(jvmOptions, debugOptions)),
+    return new YarnTwillPreparer(yarnConfig, twillSpec, yarnAppClient, zkClientService, locationFactory, jvmOptions,
                                  new YarnTwillControllerFactory() {
       @Override
       public YarnTwillController create(RunId runId, Iterable<LogHandler> logHandlers,

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
index ee98352..daf0bc1 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
@@ -104,10 +104,10 @@ public class DebugTestRun extends BaseYarnTest {
   @Test
   public void testDebugPort() throws Exception {
     YarnTwillRunnerService runner = (YarnTwillRunnerService) YarnTestUtils.getTwillRunner();
-    runner.enableDebugging("r1");
     runner.startAndWait();
 
     TwillController controller = runner.prepare(new DummyApplication())
+                                       .enableDebugging("r1")
                                        .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out)))
                                        .start();
     final CountDownLatch running = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
index 78f61a9..2bf0b5a 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
@@ -57,23 +57,14 @@ public final class LocalFileTestRun extends BaseYarnTest {
                                         Charsets.UTF_8);
 
     TwillRunner runner = YarnTestUtils.getTwillRunner();
-    String prevJVMOptions = "";
-    if (runner instanceof YarnTwillRunnerService) {
-      YarnTwillRunnerService yarnRunner = (YarnTwillRunnerService) runner;
-      prevJVMOptions = yarnRunner.getJVMOptions() != null ? yarnRunner.getJVMOptions() : "";
-      yarnRunner.setJVMOptions(prevJVMOptions + " -verbose:gc -Xloggc:gc.log -XX:+PrintGCDetails");
-    }
 
     TwillController controller = runner.prepare(new LocalFileApplication())
+      .addJVMOptions(" -verbose:gc -Xloggc:gc.log -XX:+PrintGCDetails")
       .withApplicationArguments("local")
       .withArguments("LocalFileSocketServer", "local2")
       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
       .start();
 
-    if (runner instanceof YarnTwillRunnerService) {
-      ((YarnTwillRunnerService) runner).setJVMOptions(prevJVMOptions);
-    }
-
     Iterable<Discoverable> discoverables = controller.discoverService("local");
     Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 1, 60));