You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@twill.apache.org by maochf <gi...@git.apache.org> on 2017/07/26 07:39:58 UTC

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

GitHub user maochf opened a pull request:

    https://github.com/apache/twill/pull/58

    [TWILL-240] EventHandler Improvement

    https://issues.apache.org/jira/browse/TWILL-240
    
    Add started, containerLaunched, containerStopped, completed, killed, aborted to EventHandler

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/maochf/twill feature/event-handler-improvement

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/twill/pull/58.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #58
    
----
commit 722226eeca04586712123c44caf5788c17e0968a
Author: Chengfeng <ma...@cask.co>
Date:   2017-07-25T19:53:26Z

    [TWILL-240] Add started, containerLaunched, containerStopped, completed, killed, aborted to EventHandler

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131761743
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandlerContext.java ---
    @@ -22,5 +22,9 @@
      */
     public interface EventHandlerContext {
     
    +  String getTwillAppName();
    --- End diff --
    
    Should be called `getApplicationName()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/twill/pull/58


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131210382
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -291,6 +301,23 @@ public void run() {
         // Since all the runnables are now stopped, it is okay to stop the poller.
         stopPoller.shutdownNow();
         cleanupDir();
    +    if (eventHandler != null) {
    +      if (finalStatus == null) {
    +        // if finalStatus is not set, the application must be stopped by a SystemMessages#STOP_COMMAND
    +        eventHandler.killed(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId());
    +      }
    +      switch (finalStatus) {
    --- End diff --
    
    This may have NPE. Should have a `else`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131762593
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java ---
    @@ -85,13 +105,29 @@ public TimeoutAction launchTimeout(Iterable<TimeoutEvent> timeoutEvents) {
             return TimeoutAction.recheck(10, TimeUnit.SECONDS);
           }
         }
    +
    +    @Override
    +    public void aborted() {
    +      try {
    +        new File(context.getSpecification().getConfigs().get("parentFolderPath") + File.separator
    --- End diff --
    
    When constructing file, usually not to use `File.separator`, but instead use the constructor `File(File parent, String path)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132029465
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/EventHandlerTest.java ---
    @@ -0,0 +1,351 @@
    +/*
    + * 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.Joiner;
    +import com.google.common.base.Stopwatch;
    +import com.google.common.base.Throwables;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.twill.api.AbstractTwillRunnable;
    +import org.apache.twill.api.EventHandler;
    +import org.apache.twill.api.EventHandlerContext;
    +import org.apache.twill.api.TwillApplication;
    +import org.apache.twill.api.TwillController;
    +import org.apache.twill.api.TwillSpecification;
    +import org.apache.twill.api.logging.PrinterLogHandler;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.PrintWriter;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +/**
    + * Tests {@link EventHandler} methods
    + */
    +public final class EventHandlerTest extends BaseYarnTest {
    +  private static final Logger LOG = LoggerFactory.getLogger(EventHandlerTest.class);
    +
    +  @ClassRule
    +  public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder();
    +  public static final String STARTED_FILE = "started_file";
    +  public static final String RUN_FILE = "run_file";
    +  public static final String CONTAINER_LAUNCHED_FOLDER = "launched_folder";
    +  public static final String CONTAINER_STOPPED_FOLDER = "stopped_folder";
    +  public static final String COMPLETED_FILE = "completed_file";
    +  public static final String KILLED_FILE = "killed_file";
    +  public static final String ABORTED_FILE = "aborted_file";
    +
    +  @Test
    +  public void testComplete() throws InterruptedException, ExecutionException, TimeoutException, IOException {
    +    // Create a parent folder to be written by EventHandler
    +    File parentFolder = TMP_FOLDER.newFolder();
    +    parentFolder.setWritable(true, false);
    +    TwillController controller = getTwillRunner().prepare(new CompleteApplication(parentFolder.getAbsolutePath()))
    +      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
    +      .start();
    +
    +    // Wait for the app to complete within 120 seconds.
    +    try {
    +      controller.awaitTerminated(120, TimeUnit.SECONDS);
    +      Set<String> handlerFiles = new HashSet<>(Arrays.asList(parentFolder.list()));
    +      Assert.assertEquals(5, handlerFiles.size());
    +      // EventHandler#started() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(STARTED_FILE));
    +      // CompleteRunnable#run() method should be called to create a file after EventHandler#started() method is called
    +      Assert.assertTrue(handlerFiles.contains(RUN_FILE));
    +      // EventHandler#containerLaunched(String, int, String) method should be called to create a folder
    +      Assert.assertTrue(handlerFiles.contains(CONTAINER_LAUNCHED_FOLDER));
    +      // EventHandler#containerStopped(String, int, String, int) method should be called to create a folder
    +      Assert.assertTrue(handlerFiles.contains(CONTAINER_STOPPED_FOLDER));
    +      // Assert that containerLaunched and containerStopped are called for the same containers
    +      // for the same number of times
    +      String[] containerLaunchedFiles = new File(parentFolder.getAbsolutePath(), CONTAINER_LAUNCHED_FOLDER).list();
    +      String[] containerStoppedFiles = new File(parentFolder.getAbsolutePath(), CONTAINER_STOPPED_FOLDER).list();
    +      Assert.assertEquals(containerLaunchedFiles.length, containerStoppedFiles.length);
    +      Assert.assertTrue(Arrays.asList(containerLaunchedFiles).containsAll(Arrays.asList(containerStoppedFiles)));
    +      // EventHandler#completed() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(COMPLETED_FILE));
    +    } catch (Exception e) {
    +      // kill the app as cleanup
    +      controller.kill();
    --- End diff --
    
    This is not needed. The base test class will handle it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132029508
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/EventHandlerTest.java ---
    @@ -0,0 +1,351 @@
    +/*
    + * 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.Joiner;
    +import com.google.common.base.Stopwatch;
    +import com.google.common.base.Throwables;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.twill.api.AbstractTwillRunnable;
    +import org.apache.twill.api.EventHandler;
    +import org.apache.twill.api.EventHandlerContext;
    +import org.apache.twill.api.TwillApplication;
    +import org.apache.twill.api.TwillController;
    +import org.apache.twill.api.TwillSpecification;
    +import org.apache.twill.api.logging.PrinterLogHandler;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.PrintWriter;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +/**
    + * Tests {@link EventHandler} methods
    + */
    +public final class EventHandlerTest extends BaseYarnTest {
    +  private static final Logger LOG = LoggerFactory.getLogger(EventHandlerTest.class);
    +
    +  @ClassRule
    +  public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder();
    +  public static final String STARTED_FILE = "started_file";
    +  public static final String RUN_FILE = "run_file";
    +  public static final String CONTAINER_LAUNCHED_FOLDER = "launched_folder";
    +  public static final String CONTAINER_STOPPED_FOLDER = "stopped_folder";
    +  public static final String COMPLETED_FILE = "completed_file";
    +  public static final String KILLED_FILE = "killed_file";
    +  public static final String ABORTED_FILE = "aborted_file";
    +
    +  @Test
    +  public void testComplete() throws InterruptedException, ExecutionException, TimeoutException, IOException {
    +    // Create a parent folder to be written by EventHandler
    +    File parentFolder = TMP_FOLDER.newFolder();
    +    parentFolder.setWritable(true, false);
    +    TwillController controller = getTwillRunner().prepare(new CompleteApplication(parentFolder.getAbsolutePath()))
    +      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
    +      .start();
    +
    +    // Wait for the app to complete within 120 seconds.
    +    try {
    +      controller.awaitTerminated(120, TimeUnit.SECONDS);
    +      Set<String> handlerFiles = new HashSet<>(Arrays.asList(parentFolder.list()));
    +      Assert.assertEquals(5, handlerFiles.size());
    +      // EventHandler#started() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(STARTED_FILE));
    +      // CompleteRunnable#run() method should be called to create a file after EventHandler#started() method is called
    +      Assert.assertTrue(handlerFiles.contains(RUN_FILE));
    +      // EventHandler#containerLaunched(String, int, String) method should be called to create a folder
    +      Assert.assertTrue(handlerFiles.contains(CONTAINER_LAUNCHED_FOLDER));
    +      // EventHandler#containerStopped(String, int, String, int) method should be called to create a folder
    +      Assert.assertTrue(handlerFiles.contains(CONTAINER_STOPPED_FOLDER));
    +      // Assert that containerLaunched and containerStopped are called for the same containers
    +      // for the same number of times
    +      String[] containerLaunchedFiles = new File(parentFolder.getAbsolutePath(), CONTAINER_LAUNCHED_FOLDER).list();
    +      String[] containerStoppedFiles = new File(parentFolder.getAbsolutePath(), CONTAINER_STOPPED_FOLDER).list();
    +      Assert.assertEquals(containerLaunchedFiles.length, containerStoppedFiles.length);
    +      Assert.assertTrue(Arrays.asList(containerLaunchedFiles).containsAll(Arrays.asList(containerStoppedFiles)));
    +      // EventHandler#completed() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(COMPLETED_FILE));
    +    } catch (Exception e) {
    +      // kill the app as cleanup
    +      controller.kill();
    +    }
    +  }
    +
    +  @Test
    +  public void testKilled() throws IOException, InterruptedException, TimeoutException, ExecutionException {
    +    // Create a parent folder to be written by EventHandler
    +    File parentFolder = TMP_FOLDER.newFolder();
    +    parentFolder.setWritable(true, false);
    +    TwillController controller = getTwillRunner().prepare(new SleepApplication(parentFolder.getAbsolutePath()))
    +      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
    +      .start();
    +    try {
    +      // Wait for the runnable to run and create runFile within 120 secs
    +      File runFile = new File(parentFolder, RUN_FILE);
    +      Stopwatch stopwatch = new Stopwatch().start();
    +      while (!runFile.exists() && stopwatch.elapsedTime(TimeUnit.SECONDS) < 120) {
    +        TimeUnit.SECONDS.sleep(1);
    +      }
    +      Assert.assertTrue(runFile.exists());
    +      // Terminate the app once the runnable runs
    +      controller.terminate();
    +      controller.awaitTerminated(120, TimeUnit.SECONDS);
    +      Set<String> handlerFiles = new HashSet<>(Arrays.asList(parentFolder.list()));
    +      // EventHandler#killed() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(KILLED_FILE));
    +      // EventHandler#completed() method should not be called
    +      Assert.assertFalse(handlerFiles.contains(COMPLETED_FILE));
    +      // EventHandler#aborted() method should not be called
    +      Assert.assertFalse(handlerFiles.contains(ABORTED_FILE));
    +    } catch (Exception e) {
    +      // kill the app as cleanup
    +      controller.kill();
    --- End diff --
    
    Same as above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132295847
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -181,7 +189,6 @@ public Reader getInput() throws IOException {
       }
     
       @SuppressWarnings("unchecked")
    -  @Nullable
       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();
    --- End diff --
    
    if `handlerSpec` is `null`, you still want to return a no-op handler, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132322087
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -181,19 +190,130 @@ public Reader getInput() throws IOException {
       }
     
       @SuppressWarnings("unchecked")
    -  @Nullable
       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;
    +      // if no handler is specified, return an EventHandler with no-op
    +      return new EventHandler() {};
         }
     
         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);
    +    final EventHandler delegate = Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    // wrap all calls to the delegate EventHandler methods except initialize so that all errors will be caught
    +    return new EventHandler() {
    +      private boolean initialized;
    +
    +      @Override
    +      public void initialize(EventHandlerContext context) {
    +        delegate.initialize(context);
    +        initialized = true;
    --- End diff --
    
    Since any exception thrown from `initialize` will terminate the app, I think we don't need this, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132296875
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -193,7 +200,91 @@ private EventHandler createEventHandler(TwillSpecification twillSpec) throws Cla
         Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
                                     "Class {} does not implements {}",
                                     handlerClass, EventHandler.class.getName());
    -    return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    final EventHandler delegate = Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    if (delegate == null) {
    +      // if no handler is specified, return an EventHandler with no-op
    +      return new EventHandler() {};
    +    }
    +    // wrap the delegate EventHandler so that all errors will be caught
    +    return new EventHandler() {
    +
    +      @Override
    +      public void started() {
    +        try {
    +          delegate.started();
    +        } catch (Throwable t) {
    +          LOG.warn("Exception when calling {}.started()", eventHandler.getClass().getName(), t);
    --- End diff --
    
    `Exception raised when...`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132322898
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -239,14 +359,8 @@ protected void doStop() throws Exception {
         LOG.info("Stop application master with spec: {}",
                  TwillRuntimeSpecificationAdapter.create().toJson(twillRuntimeSpec));
     
    -    if (eventHandler != null) {
    -      try {
    -        // call event handler destroy. If there is error, only log and not affected stop sequence.
    -        eventHandler.destroy();
    -      } catch (Throwable t) {
    -        LOG.warn("Exception when calling {}.destroy()", eventHandler.getClass().getName(), t);
    -      }
    -    }
    +    // call event handler destroy
    +    eventHandler.destroy();
    --- End diff --
    
    I think so.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132297305
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -193,7 +200,91 @@ private EventHandler createEventHandler(TwillSpecification twillSpec) throws Cla
         Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
                                     "Class {} does not implements {}",
                                     handlerClass, EventHandler.class.getName());
    -    return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    final EventHandler delegate = Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    if (delegate == null) {
    +      // if no handler is specified, return an EventHandler with no-op
    +      return new EventHandler() {};
    +    }
    +    // wrap the delegate EventHandler so that all errors will be caught
    +    return new EventHandler() {
    --- End diff --
    
    You'll need to wrap the `initialize` as well. Also, I'd suggest if `initialize` throw exception, you'll just ignore all calls to other lifecycle methods to avoid excessive logging.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132040768
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/EventHandlerTest.java ---
    @@ -0,0 +1,351 @@
    +/*
    + * 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.Joiner;
    +import com.google.common.base.Stopwatch;
    +import com.google.common.base.Throwables;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.twill.api.AbstractTwillRunnable;
    +import org.apache.twill.api.EventHandler;
    +import org.apache.twill.api.EventHandlerContext;
    +import org.apache.twill.api.TwillApplication;
    +import org.apache.twill.api.TwillController;
    +import org.apache.twill.api.TwillSpecification;
    +import org.apache.twill.api.logging.PrinterLogHandler;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.PrintWriter;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +/**
    + * Tests {@link EventHandler} methods
    + */
    +public final class EventHandlerTest extends BaseYarnTest {
    +  private static final Logger LOG = LoggerFactory.getLogger(EventHandlerTest.class);
    +
    +  @ClassRule
    +  public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder();
    +  public static final String STARTED_FILE = "started_file";
    +  public static final String RUN_FILE = "run_file";
    +  public static final String CONTAINER_LAUNCHED_FOLDER = "launched_folder";
    +  public static final String CONTAINER_STOPPED_FOLDER = "stopped_folder";
    +  public static final String COMPLETED_FILE = "completed_file";
    +  public static final String KILLED_FILE = "killed_file";
    +  public static final String ABORTED_FILE = "aborted_file";
    +
    +  @Test
    +  public void testComplete() throws InterruptedException, ExecutionException, TimeoutException, IOException {
    +    // Create a parent folder to be written by EventHandler
    +    File parentFolder = TMP_FOLDER.newFolder();
    +    parentFolder.setWritable(true, false);
    +    TwillController controller = getTwillRunner().prepare(new CompleteApplication(parentFolder.getAbsolutePath()))
    +      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
    +      .start();
    +
    +    // Wait for the app to complete within 120 seconds.
    +    try {
    +      controller.awaitTerminated(120, TimeUnit.SECONDS);
    +      Set<String> handlerFiles = new HashSet<>(Arrays.asList(parentFolder.list()));
    +      Assert.assertEquals(5, handlerFiles.size());
    +      // EventHandler#started() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(STARTED_FILE));
    +      // CompleteRunnable#run() method should be called to create a file after EventHandler#started() method is called
    +      Assert.assertTrue(handlerFiles.contains(RUN_FILE));
    +      // EventHandler#containerLaunched(String, int, String) method should be called to create a folder
    +      Assert.assertTrue(handlerFiles.contains(CONTAINER_LAUNCHED_FOLDER));
    +      // EventHandler#containerStopped(String, int, String, int) method should be called to create a folder
    +      Assert.assertTrue(handlerFiles.contains(CONTAINER_STOPPED_FOLDER));
    +      // Assert that containerLaunched and containerStopped are called for the same containers
    +      // for the same number of times
    +      String[] containerLaunchedFiles = new File(parentFolder.getAbsolutePath(), CONTAINER_LAUNCHED_FOLDER).list();
    +      String[] containerStoppedFiles = new File(parentFolder.getAbsolutePath(), CONTAINER_STOPPED_FOLDER).list();
    +      Assert.assertEquals(containerLaunchedFiles.length, containerStoppedFiles.length);
    +      Assert.assertTrue(Arrays.asList(containerLaunchedFiles).containsAll(Arrays.asList(containerStoppedFiles)));
    +      // EventHandler#completed() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(COMPLETED_FILE));
    +    } catch (Exception e) {
    +      // kill the app as cleanup
    +      controller.kill();
    +    }
    +  }
    +
    +  @Test
    +  public void testKilled() throws IOException, InterruptedException, TimeoutException, ExecutionException {
    +    // Create a parent folder to be written by EventHandler
    +    File parentFolder = TMP_FOLDER.newFolder();
    +    parentFolder.setWritable(true, false);
    +    TwillController controller = getTwillRunner().prepare(new SleepApplication(parentFolder.getAbsolutePath()))
    +      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
    +      .start();
    +    try {
    +      // Wait for the runnable to run and create runFile within 120 secs
    +      File runFile = new File(parentFolder, RUN_FILE);
    +      Stopwatch stopwatch = new Stopwatch().start();
    +      while (!runFile.exists() && stopwatch.elapsedTime(TimeUnit.SECONDS) < 120) {
    +        TimeUnit.SECONDS.sleep(1);
    +      }
    +      Assert.assertTrue(runFile.exists());
    +      // Terminate the app once the runnable runs
    +      controller.terminate();
    +      controller.awaitTerminated(120, TimeUnit.SECONDS);
    +      Set<String> handlerFiles = new HashSet<>(Arrays.asList(parentFolder.list()));
    +      // EventHandler#killed() method should be called to create a file
    +      Assert.assertTrue(handlerFiles.contains(KILLED_FILE));
    +      // EventHandler#completed() method should not be called
    +      Assert.assertFalse(handlerFiles.contains(COMPLETED_FILE));
    +      // EventHandler#aborted() method should not be called
    +      Assert.assertFalse(handlerFiles.contains(ABORTED_FILE));
    +    } catch (Exception e) {
    +      // kill the app as cleanup
    +      controller.kill();
    +    }
    +  }
    +
    +  /**
    +   * The handler for testing timeout handling.
    +   */
    +  public static final class Handler extends EventHandler {
    +
    +    private final String parentFolderPath;
    +    private boolean abort;
    +
    +    public Handler(String parentFolderPath) {
    +      this.parentFolderPath = parentFolderPath;
    +    }
    +
    +    @Override
    +    protected Map<String, String> getConfigs() {
    +      return ImmutableMap.<String, String>builder()
    +        .put("abort", "true")
    +        .put("parentFolderPath", parentFolderPath)
    +        .put("startedFile", STARTED_FILE)
    +        .put("runFile", RUN_FILE)
    +        .put("containerLaunched", CONTAINER_LAUNCHED_FOLDER)
    +        .put("containerStopped", CONTAINER_STOPPED_FOLDER)
    +        .put("completedFile", COMPLETED_FILE)
    +        .put("killedFile", KILLED_FILE)
    +        .build();
    +    }
    +
    +    @Override
    +    public void initialize(EventHandlerContext context) {
    +      super.initialize(context);
    +      this.abort = Boolean.parseBoolean(context.getSpecification().getConfigs().get("abort"));
    +    }
    +
    +    @Override
    +    public TimeoutAction launchTimeout(Iterable<TimeoutEvent> timeoutEvents) {
    +      if (abort) {
    +        return TimeoutAction.abort();
    +      } else {
    +        return TimeoutAction.recheck(10, TimeUnit.SECONDS);
    +      }
    +    }
    +
    +    @Override
    +    public void started() {
    +      try {
    +        new File(context.getSpecification().getConfigs().get("parentFolderPath"),
    +                 context.getSpecification().getConfigs().get("startedFile")).createNewFile();
    +      } catch (IOException e) {
    +        Throwables.propagate(e);
    +      }
    +    }
    +
    +    @Override
    +    public void containerLaunched(String runnableName, int instanceId, String containerId) {
    +      LOG.info("Launched {}#{} in container {}", runnableName, instanceId, containerId);
    +      createContainerFile(runnableName, instanceId, containerId, "containerLaunched");
    +    }
    +
    +    @Override
    +    public void containerStopped(String runnableName, int instanceId, String containerId, int exitStatus) {
    +      LOG.info("Stopped {}#{} in container {} with status {}", runnableName, instanceId, containerId, exitStatus);
    +      createContainerFile(runnableName, instanceId, containerId, "containerStopped");
    +    }
    +
    +    private void createContainerFile(String runnableName, int instanceId, String containerId, String folderKey) {
    +      Map<String, String> configs = context.getSpecification().getConfigs();
    +      File launchedFolder = new File(configs.get("parentFolderPath"), configs.get(folderKey));
    +      if (!launchedFolder.exists()) {
    +        launchedFolder.mkdirs();
    +        launchedFolder.setReadable(true, false);
    +      }
    +      try {
    +        new File(launchedFolder.getAbsolutePath(), Joiner.on(":").join(runnableName, instanceId, containerId))
    --- End diff --
    
    Would be clear if you have a private `touch(String name)` method rather than have `try-catch` block on each callback method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132297954
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java ---
    @@ -401,6 +409,24 @@ void stopAll() {
         // When we acquire this lock, all stopped runnables should have been cleaned up by handleCompleted() method
         containerLock.lock();
         try {
    +      for (Map.Entry<String, Map<String, TwillContainerController>> entry : containers.rowMap().entrySet()) {
    +        String runnableName = entry.getKey();
    +        Collection<ContainerInfo> containerInfos = containerStats.get(runnableName);
    +        for (Map.Entry<String, TwillContainerController> containerControllerEntry : entry.getValue().entrySet()) {
    +          boolean containerExist = false;
    +          for (ContainerInfo containerInfo : containerInfos) {
    +            if (containerInfo.getId().equals(containerControllerEntry.getKey())) {
    +              containerExist = true;
    +              break;
    +            }
    +          }
    +          // Only call eventHandler.containerStopped if container is not removed by handleCompleted
    +          if (containerExist) {
    +            eventHandler.containerStopped(runnableName, containerControllerEntry.getValue().getInstanceId(),
    --- End diff --
    
    you can move this inside the `for` loop and no need to use `containerExist`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by sameetandpotatoes <gi...@git.apache.org>.
Github user sameetandpotatoes commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r129645453
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -222,6 +226,7 @@ protected void doStart() throws Exception {
         // initialize the event handler, if it fails, it will fail the application.
         if (eventHandler != null) {
           eventHandler.initialize(new BasicEventHandlerContext(twillSpec.getEventHandler()));
    +      eventHandler.started(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId());
    --- End diff --
    
    According to the comment above this `if` statement, if the event handler fails to initialize, will any twill application state be recorded?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131211061
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandler.java ---
    @@ -124,6 +124,75 @@ public void initialize(EventHandlerContext context) {
       }
     
       /**
    +   * Invoked by the application when it starts.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void started(String twillAppName, RunId runId) {
    --- End diff --
    
    Why all methods has the app name and runid? It should be provided via the `EventHandlerContext` since it will never change for an application run.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by sameetandpotatoes <gi...@git.apache.org>.
Github user sameetandpotatoes commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r129643180
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java ---
    @@ -103,6 +105,8 @@ public Integer apply(BitSet input) {
       private final Table<String, String, TwillContainerController> containers;
     
       // Map from runnableName to a BitSet, with the <instanceId> bit turned on for having an instance running.
    +  private final String twillAppName;
    --- End diff --
    
    The comment above this line should be moved down to above the `runnableInstances` variable


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132040093
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java ---
    @@ -267,6 +280,14 @@ void stopByIdAndWait(String runnableName, int instanceId) {
     
             resourceReport.removeRunnableResources(runnableName, containerId);
             containerChange.signalAll();
    +        if (eventHandler != null) {
    +          Integer exitStatus = containerExitStatus.get(containerId);
    +          if (exitStatus == null) {
    --- End diff --
    
    So this is for handling the case when `handleCompleted` was not called (e.g. the runnable container is not stopping after receiving the "stop" command), right? If that's the case, isn't that the exit status is always not there?
    
    Also, do we need to do the similar logic to call event handler in the `stopAll` method?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill issue #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on the issue:

    https://github.com/apache/twill/pull/58
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131210237
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -291,6 +301,23 @@ public void run() {
         // Since all the runnables are now stopped, it is okay to stop the poller.
         stopPoller.shutdownNow();
         cleanupDir();
    +    if (eventHandler != null) {
    +      if (finalStatus == null) {
    +        // if finalStatus is not set, the application must be stopped by a SystemMessages#STOP_COMMAND
    +        eventHandler.killed(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId());
    +      }
    +      switch (finalStatus) {
    +        case COMPLETED:
    +          eventHandler.completed(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId());
    +          break;
    +        case ABORTED:
    +        eventHandler.aborted(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId());
    +          break;
    +        default:
    +          // should never reach here
    +          LOG.error("Unsupported FinalStatus '%s'", finalStatus.name());
    --- End diff --
    
    That's not the right syntax. slf4j logger uses `{}`. Should be `LOG.error("Unsupported status {}", finalStatus)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131225251
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java ---
    @@ -79,12 +83,22 @@ public void initialize(EventHandlerContext context) {
     
         @Override
         public TimeoutAction launchTimeout(Iterable<TimeoutEvent> timeoutEvents) {
    +      for (TimeoutEvent event : timeoutEvents) {
    +        LOG.info("Requested {} containers for runnable {}, only got {} after {} ms.",
    --- End diff --
    
    i was using it to confirm the change. Will remove


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131761787
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandlerContext.java ---
    @@ -22,5 +22,9 @@
      */
     public interface EventHandlerContext {
     
    +  String getTwillAppName();
    +
    +  String getTwillAppRunId();
    --- End diff --
    
    Just call it `getRunId()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132296198
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -193,7 +200,91 @@ private EventHandler createEventHandler(TwillSpecification twillSpec) throws Cla
         Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
                                     "Class {} does not implements {}",
                                     handlerClass, EventHandler.class.getName());
    -    return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    final EventHandler delegate = Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    if (delegate == null) {
    --- End diff --
    
    `Instances.newInstance` never return `null`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r129647384
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -222,6 +226,7 @@ protected void doStart() throws Exception {
         // initialize the event handler, if it fails, it will fail the application.
         if (eventHandler != null) {
           eventHandler.initialize(new BasicEventHandlerContext(twillSpec.getEventHandler()));
    +      eventHandler.started(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId());
    --- End diff --
    
    If initialization fails, then the whole app will fail. `TwillController` should be able to catch this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r130406951
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandler.java ---
    @@ -124,6 +124,74 @@ public void initialize(EventHandlerContext context) {
       }
     
       /**
    +   * Invoked by the application when it starts.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void started(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when new container is launched for {@link TwillRunnable}.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable to be run in the new container
    +   * @param instanceId the instance ID of the runnable instance to be run in the new container
    +   * @param containerId the ID of the newly launched container
    +   */
    +  public void containerLaunched(String twillAppName, RunId runId, String runnableName,
    +                                int instanceId, String containerId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when a container is stopped.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable in the stopped container
    +   * @param instanceId the instance ID of the runnable instance run in the stopped container
    +   * @param containerId the ID of the stopped container
    +   */
    +  public void containerStopped(String twillAppName, RunId runId, String runnableName,
    --- End diff --
    
    Should provide the exit code as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r130491020
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandler.java ---
    @@ -124,6 +124,74 @@ public void initialize(EventHandlerContext context) {
       }
     
       /**
    +   * Invoked by the application when it starts.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void started(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when new container is launched for {@link TwillRunnable}.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable to be run in the new container
    +   * @param instanceId the instance ID of the runnable instance to be run in the new container
    +   * @param containerId the ID of the newly launched container
    +   */
    +  public void containerLaunched(String twillAppName, RunId runId, String runnableName,
    +                                int instanceId, String containerId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when a container is stopped.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable in the stopped container
    +   * @param instanceId the instance ID of the runnable instance run in the stopped container
    +   * @param containerId the ID of the stopped container
    +   */
    +  public void containerStopped(String twillAppName, RunId runId, String runnableName,
    +                               int instanceId, String containerId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when all containers complete.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void completed(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when stop command is received to kill the current application.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void killed(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when the application is aborted because of timeout.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void aborted(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
        * Invoked by the application when shutting down.
        */
       public void destroy() {
    --- End diff --
    
    what should be the default action of `launchTimeout`? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131208560
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandler.java ---
    @@ -124,6 +124,74 @@ public void initialize(EventHandlerContext context) {
       }
     
       /**
    +   * Invoked by the application when it starts.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void started(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when new container is launched for {@link TwillRunnable}.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable to be run in the new container
    +   * @param instanceId the instance ID of the runnable instance to be run in the new container
    +   * @param containerId the ID of the newly launched container
    +   */
    +  public void containerLaunched(String twillAppName, RunId runId, String runnableName,
    +                                int instanceId, String containerId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when a container is stopped.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable in the stopped container
    +   * @param instanceId the instance ID of the runnable instance run in the stopped container
    +   * @param containerId the ID of the stopped container
    +   */
    +  public void containerStopped(String twillAppName, RunId runId, String runnableName,
    +                               int instanceId, String containerId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when all containers complete.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void completed(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when stop command is received to kill the current application.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void killed(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when the application is aborted because of timeout.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void aborted(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
        * Invoked by the application when shutting down.
        */
       public void destroy() {
    --- End diff --
    
    continue, with the recheck time to some hardcode constant (say 60 seconds).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131211956
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java ---
    @@ -79,12 +83,22 @@ public void initialize(EventHandlerContext context) {
     
         @Override
         public TimeoutAction launchTimeout(Iterable<TimeoutEvent> timeoutEvents) {
    +      for (TimeoutEvent event : timeoutEvents) {
    +        LOG.info("Requested {} containers for runnable {}, only got {} after {} ms.",
    --- End diff --
    
    What's the point of this logging? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132038521
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java ---
    @@ -267,6 +280,14 @@ void stopByIdAndWait(String runnableName, int instanceId) {
     
             resourceReport.removeRunnableResources(runnableName, containerId);
             containerChange.signalAll();
    +        if (eventHandler != null) {
    +          Integer exitStatus = containerExitStatus.get(containerId);
    --- End diff --
    
    Should remove it from the exit status map, otherwise the map can keep growing in size.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131762365
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandlerContext.java ---
    @@ -22,5 +22,9 @@
      */
     public interface EventHandlerContext {
     
    +  String getTwillAppName();
    +
    +  String getTwillAppRunId();
    --- End diff --
    
    The run id should be of type `RunId`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131761843
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandlerContext.java ---
    @@ -22,5 +22,9 @@
      */
     public interface EventHandlerContext {
     
    +  String getTwillAppName();
    --- End diff --
    
    Also please add javadoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132322316
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -239,14 +359,8 @@ protected void doStop() throws Exception {
         LOG.info("Stop application master with spec: {}",
                  TwillRuntimeSpecificationAdapter.create().toJson(twillRuntimeSpec));
     
    -    if (eventHandler != null) {
    -      try {
    -        // call event handler destroy. If there is error, only log and not affected stop sequence.
    -        eventHandler.destroy();
    -      } catch (Throwable t) {
    -        LOG.warn("Exception when calling {}.destroy()", eventHandler.getClass().getName(), t);
    -      }
    -    }
    +    // call event handler destroy
    +    eventHandler.destroy();
    --- End diff --
    
    Shouldn't call `destroy()` here anymore. It should be called after call other calls to the event handler, as it is acted as a cleanup call.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131234842
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandler.java ---
    @@ -124,6 +124,75 @@ public void initialize(EventHandlerContext context) {
       }
     
       /**
    +   * Invoked by the application when it starts.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void started(String twillAppName, RunId runId) {
    --- End diff --
    
    the `EventHandlerContext` just contains a `EventHandlerSpecification`, and it doesn't seem to guarantee app name and runid are there?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132298929
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -193,7 +200,91 @@ private EventHandler createEventHandler(TwillSpecification twillSpec) throws Cla
         Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
                                     "Class {} does not implements {}",
                                     handlerClass, EventHandler.class.getName());
    -    return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    final EventHandler delegate = Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    if (delegate == null) {
    +      // if no handler is specified, return an EventHandler with no-op
    +      return new EventHandler() {};
    +    }
    +    // wrap the delegate EventHandler so that all errors will be caught
    +    return new EventHandler() {
    --- End diff --
    
    hum.. that's true.. maybe we should keep it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132038164
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java ---
    @@ -173,6 +183,9 @@ void start(String runnableName, ContainerInfo containerInfo, TwillContainerLaunc
             startSequence.addLast(runnableName);
           }
           containerChange.signalAll();
    +      if (eventHandler != null) {
    +        eventHandler.containerLaunched(runnableName, instanceId, containerInfo.getId());
    --- End diff --
    
    How do we handle exception raised from event handler methods?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r130407224
  
    --- Diff: twill-api/src/main/java/org/apache/twill/api/EventHandler.java ---
    @@ -124,6 +124,74 @@ public void initialize(EventHandlerContext context) {
       }
     
       /**
    +   * Invoked by the application when it starts.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void started(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when new container is launched for {@link TwillRunnable}.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable to be run in the new container
    +   * @param instanceId the instance ID of the runnable instance to be run in the new container
    +   * @param containerId the ID of the newly launched container
    +   */
    +  public void containerLaunched(String twillAppName, RunId runId, String runnableName,
    +                                int instanceId, String containerId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when a container is stopped.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   * @param runnableName name of the runnable in the stopped container
    +   * @param instanceId the instance ID of the runnable instance run in the stopped container
    +   * @param containerId the ID of the stopped container
    +   */
    +  public void containerStopped(String twillAppName, RunId runId, String runnableName,
    +                               int instanceId, String containerId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when all containers complete.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void completed(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when stop command is received to kill the current application.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void killed(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
    +   * Invoked by the application when the application is aborted because of timeout.
    +   *
    +   * @param twillAppName name of the current application
    +   * @param runId run ID of current application run
    +   */
    +  public void aborted(String twillAppName, RunId runId) {
    +    // No-op
    +  }
    +
    +  /**
        * Invoked by the application when shutting down.
        */
       public void destroy() {
    --- End diff --
    
    Can't add comment at line 206, but it's better to make the `launchTimeout` method non-abstract.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132037568
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -221,7 +229,8 @@ protected void doStart() throws Exception {
     
         // initialize the event handler, if it fails, it will fail the application.
         if (eventHandler != null) {
    -      eventHandler.initialize(new BasicEventHandlerContext(twillSpec.getEventHandler()));
    +      eventHandler.initialize(new BasicEventHandlerContext(twillRuntimeSpec, twillSpec.getEventHandler()));
    --- End diff --
    
    The `twillRuntimeSpec` already contains `twillSpec`. Seems unnecessary to pass in `twillSpec.getEventHandler()` as a separate parameter.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill issue #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on the issue:

    https://github.com/apache/twill/pull/58
  
    squashed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r130407739
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -205,8 +208,9 @@ private RunningContainers createRunningContainers(ContainerId appMasterContainer
           Integer.parseInt(System.getenv(EnvKeys.YARN_CONTAINER_MEMORY_MB)),
           appMasterHost, null);
         String appId = appMasterContainerId.getApplicationAttemptId().getApplicationId().toString();
    -    return new RunningContainers(appId, appMasterResources, zkClient, applicationLocation,
    -      twillSpec.getRunnables(), twillRuntimeSpec.getMaxRetries());
    +    return new RunningContainers(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId(),
    --- End diff --
    
    Better just pass the `twillRuntimeSpec` to the `RunningContainers` instead of getting individual entries one by one. It can replace the `twillRuntimeSpec.getXXX` and `twillSpec.getRunnables`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131209163
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -135,6 +135,15 @@
       private final Map<String, Map<String, String>> environments;
       private final TwillRuntimeSpecification twillRuntimeSpec;
     
    +  /**
    +   * Final status of this service when it stops.
    +   */
    +  public enum FinalStatus {
    --- End diff --
    
    Sounds more like `StopStatus` or `CompletionStatus`. Also it should be `private` since it is only used in this class.
    
    Also, please move it before all fields declaration instead of having it in between fields.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131762894
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java ---
    @@ -85,13 +105,29 @@ public TimeoutAction launchTimeout(Iterable<TimeoutEvent> timeoutEvents) {
             return TimeoutAction.recheck(10, TimeUnit.SECONDS);
           }
         }
    +
    +    @Override
    +    public void aborted() {
    +      try {
    +        new File(context.getSpecification().getConfigs().get("parentFolderPath") + File.separator
    --- End diff --
    
    Also, we are only testing abort? How about other lifecycle methods?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132298585
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -193,7 +200,91 @@ private EventHandler createEventHandler(TwillSpecification twillSpec) throws Cla
         Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
                                     "Class {} does not implements {}",
                                     handlerClass, EventHandler.class.getName());
    -    return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    final EventHandler delegate = Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    if (delegate == null) {
    +      // if no handler is specified, return an EventHandler with no-op
    +      return new EventHandler() {};
    +    }
    +    // wrap the delegate EventHandler so that all errors will be caught
    +    return new EventHandler() {
    --- End diff --
    
    originally, if `EventHandler#initialize` fails, the app will fail to start. Do we want to change this behavior?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132048851
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java ---
    @@ -267,6 +280,14 @@ void stopByIdAndWait(String runnableName, int instanceId) {
     
             resourceReport.removeRunnableResources(runnableName, containerId);
             containerChange.signalAll();
    +        if (eventHandler != null) {
    +          Integer exitStatus = containerExitStatus.get(containerId);
    +          if (exitStatus == null) {
    --- End diff --
    
    `controller.stopAndWait();` at line 269 blocks until `handleCompleted` method runs


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by sameetandpotatoes <gi...@git.apache.org>.
Github user sameetandpotatoes commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r129643121
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java ---
    @@ -267,6 +281,9 @@ void stopByIdAndWait(String runnableName, int instanceId) {
     
             resourceReport.removeRunnableResources(runnableName, containerId);
             containerChange.signalAll();
    +        if (eventHandler != null) {
    +          eventHandler.containerLaunched(twillAppName, runId, runnableName, instanceId, containerId);
    --- End diff --
    
    Typo - should be `containerStopped`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131212239
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java ---
    @@ -79,12 +83,22 @@ public void initialize(EventHandlerContext context) {
     
         @Override
         public TimeoutAction launchTimeout(Iterable<TimeoutEvent> timeoutEvents) {
    +      for (TimeoutEvent event : timeoutEvents) {
    +        LOG.info("Requested {} containers for runnable {}, only got {} after {} ms.",
    +                 event.getExpectedInstances(), event.getRunnableName(),
    +                 event.getActualInstances(), System.currentTimeMillis() - event.getRequestTime());
    +      }
           if (abort) {
             return TimeoutAction.abort();
           } else {
             return TimeoutAction.recheck(10, TimeUnit.SECONDS);
           }
         }
    +
    +    @Override
    +    public void aborted(String twillAppName, RunId runId) {
    +      LOG.info(String.format("Aborted %s with runId %s", twillAppName, runId.getId()));
    --- End diff --
    
    Besides logging, can you add/modify the test to actually validate the new lifecycle methods in the EventHandler?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by maochf <gi...@git.apache.org>.
Github user maochf commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132322804
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -239,14 +359,8 @@ protected void doStop() throws Exception {
         LOG.info("Stop application master with spec: {}",
                  TwillRuntimeSpecificationAdapter.create().toJson(twillRuntimeSpec));
     
    -    if (eventHandler != null) {
    -      try {
    -        // call event handler destroy. If there is error, only log and not affected stop sequence.
    -        eventHandler.destroy();
    -      } catch (Throwable t) {
    -        LOG.warn("Exception when calling {}.destroy()", eventHandler.getClass().getName(), t);
    -      }
    -    }
    +    // call event handler destroy
    +    eventHandler.destroy();
    --- End diff --
    
    call it at the end of `doStop() `?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132029365
  
    --- Diff: twill-yarn/src/test/java/org/apache/twill/yarn/EventHandlerTest.java ---
    @@ -0,0 +1,351 @@
    +/*
    + * 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.Joiner;
    +import com.google.common.base.Stopwatch;
    +import com.google.common.base.Throwables;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.twill.api.AbstractTwillRunnable;
    +import org.apache.twill.api.EventHandler;
    +import org.apache.twill.api.EventHandlerContext;
    +import org.apache.twill.api.TwillApplication;
    +import org.apache.twill.api.TwillController;
    +import org.apache.twill.api.TwillSpecification;
    +import org.apache.twill.api.logging.PrinterLogHandler;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.PrintWriter;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +/**
    + * Tests {@link EventHandler} methods
    + */
    +public final class EventHandlerTest extends BaseYarnTest {
    --- End diff --
    
    Rename the test class to `EventHandlerTestRun` and add it to `YarnTestSuite`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r132296480
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -193,7 +200,91 @@ private EventHandler createEventHandler(TwillSpecification twillSpec) throws Cla
         Preconditions.checkArgument(EventHandler.class.isAssignableFrom(handlerClass),
                                     "Class {} does not implements {}",
                                     handlerClass, EventHandler.class.getName());
    -    return Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    final EventHandler delegate = Instances.newInstance((Class<? extends EventHandler>) handlerClass);
    +    if (delegate == null) {
    +      // if no handler is specified, return an EventHandler with no-op
    +      return new EventHandler() {};
    +    }
    +    // wrap the delegate EventHandler so that all errors will be caught
    +    return new EventHandler() {
    +
    +      @Override
    +      public void started() {
    +        try {
    +          delegate.started();
    +        } catch (Throwable t) {
    +          LOG.warn("Exception when calling {}.started()", eventHandler.getClass().getName(), t);
    --- End diff --
    
    Shouldn't it be `delegate.getClass().getName()`??


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r130407371
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -135,6 +135,9 @@
       private final Map<String, Map<String, String>> environments;
       private final TwillRuntimeSpecification twillRuntimeSpec;
     
    +  private volatile boolean completed;
    --- End diff --
    
    Instead of having three variables for `completed`, `aborted`, `stopped`, it's better to have one Enum to represent them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] twill pull request #58: [TWILL-240] EventHandler Improvement

Posted by chtyim <gi...@git.apache.org>.
Github user chtyim commented on a diff in the pull request:

    https://github.com/apache/twill/pull/58#discussion_r131762189
  
    --- Diff: twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java ---
    @@ -291,6 +300,24 @@ public void run() {
         // Since all the runnables are now stopped, it is okay to stop the poller.
         stopPoller.shutdownNow();
         cleanupDir();
    +    if (eventHandler != null) {
    +      if (stopStatus == null) {
    +        // if finalStatus is not set, the application must be stopped by a SystemMessages#STOP_COMMAND
    +        eventHandler.killed(twillRuntimeSpec.getTwillAppName(), twillRuntimeSpec.getTwillAppRunId());
    +        return;
    +      }
    +      switch (stopStatus) {
    +        case COMPLETED:
    +          eventHandler.completed();
    +          break;
    +        case ABORTED:
    +        eventHandler.aborted();
    --- End diff --
    
    Misalignment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---