You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2014/04/30 17:46:46 UTC

git commit: Adding unhandled error logging into AsyncModule.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 97cfd1ee0 -> 843d19da7


Adding unhandled error logging into AsyncModule.

Bugs closed: AURORA-366

Reviewed at https://reviews.apache.org/r/20808/


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

Branch: refs/heads/master
Commit: 843d19da7abd3b87756ee6a1bbff71bdd1905d6d
Parents: 97cfd1e
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed Apr 30 08:46:19 2014 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed Apr 30 08:46:19 2014 -0700

----------------------------------------------------------------------
 .../aurora/scheduler/SchedulerModule.java       | 12 +--
 .../aurora/scheduler/async/AsyncModule.java     |  8 +-
 .../aurora/scheduler/async/TaskGroups.java      | 13 +--
 .../apache/aurora/scheduler/base/AsyncUtil.java | 94 ++++++++++++++++++++
 .../apache/aurora/scheduler/sla/SlaModule.java  | 36 ++------
 .../scheduler/stats/AsyncStatsModule.java       | 10 ++-
 .../scheduler/storage/log/LogStorage.java       |  4 +-
 .../aurora/scheduler/base/AsyncUtilTest.java    | 87 ++++++++++++++++++
 8 files changed, 212 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/main/java/org/apache/aurora/scheduler/SchedulerModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/SchedulerModule.java b/src/main/java/org/apache/aurora/scheduler/SchedulerModule.java
index 0092372..8d1c13c 100644
--- a/src/main/java/org/apache/aurora/scheduler/SchedulerModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/SchedulerModule.java
@@ -16,13 +16,12 @@
 package org.apache.aurora.scheduler;
 
 import java.util.List;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.logging.Logger;
 
 import javax.inject.Singleton;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.inject.AbstractModule;
 import com.google.inject.PrivateModule;
 import com.google.inject.Provides;
@@ -36,6 +35,7 @@ import org.apache.aurora.scheduler.Driver.SettableDriver;
 import org.apache.aurora.scheduler.SchedulerLifecycle.LeadingOptions;
 import org.apache.aurora.scheduler.TaskIdGenerator.TaskIdGeneratorImpl;
 import org.apache.aurora.scheduler.async.GcExecutorLauncher;
+import org.apache.aurora.scheduler.base.AsyncUtil;
 import org.apache.aurora.scheduler.events.PubsubEventModule;
 import org.apache.mesos.Scheduler;
 
@@ -44,6 +44,8 @@ import org.apache.mesos.Scheduler;
  */
 public class SchedulerModule extends AbstractModule {
 
+  private static final Logger LOG = Logger.getLogger(SchedulerModule.class.getName());
+
   @CmdLine(name = "max_registration_delay",
       help = "Max allowable delay to allow the driver to register before aborting")
   private static final Arg<Amount<Long, Time>> MAX_REGISTRATION_DELAY =
@@ -73,9 +75,9 @@ public class SchedulerModule extends AbstractModule {
         bind(LeadingOptions.class).toInstance(
             new LeadingOptions(MAX_REGISTRATION_DELAY.get(), MAX_LEADING_DURATION.get()));
 
-        final ScheduledExecutorService executor = Executors.newScheduledThreadPool(
-            1,
-            new ThreadFactoryBuilder().setNameFormat("Lifecycle-%d").setDaemon(true).build());
+        final ScheduledExecutorService executor =
+            AsyncUtil.singleThreadLoggingScheduledExecutor("Lifecycle-%d", LOG);
+
         bind(ScheduledExecutorService.class).toInstance(executor);
         bind(SchedulerLifecycle.class).in(Singleton.class);
         expose(SchedulerLifecycle.class);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java b/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java
index 535acd2..834a8c8 100644
--- a/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java
@@ -27,7 +27,6 @@ import javax.inject.Singleton;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.RateLimiter;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.inject.AbstractModule;
 import com.google.inject.Binder;
 import com.google.inject.BindingAnnotation;
@@ -53,6 +52,7 @@ import org.apache.aurora.scheduler.async.OfferQueue.OfferReturnDelay;
 import org.apache.aurora.scheduler.async.RescheduleCalculator.RescheduleCalculatorImpl;
 import org.apache.aurora.scheduler.async.TaskGroups.TaskGroupsSettings;
 import org.apache.aurora.scheduler.async.TaskScheduler.TaskSchedulerImpl;
+import org.apache.aurora.scheduler.base.AsyncUtil;
 import org.apache.aurora.scheduler.events.PubsubEventModule;
 import org.apache.aurora.scheduler.filter.AttributeAggregate;
 
@@ -177,9 +177,9 @@ public class AsyncModule extends AbstractModule {
   @Override
   protected void configure() {
     // Don't worry about clean shutdown, these can be daemon and cleanup-free.
-    final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(
-        ASYNC_WORKER_THREADS.get(),
-        new ThreadFactoryBuilder().setNameFormat("AsyncProcessor-%d").setDaemon(true).build());
+    final ScheduledThreadPoolExecutor executor =
+        AsyncUtil.loggingScheduledExecutor(ASYNC_WORKER_THREADS.get(), "AsyncProcessor-%d", LOG);
+
     Stats.exportSize("timeout_queue_size", executor.getQueue());
     Stats.export(new StatImpl<Long>("async_tasks_completed") {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/main/java/org/apache/aurora/scheduler/async/TaskGroups.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/async/TaskGroups.java b/src/main/java/org/apache/aurora/scheduler/async/TaskGroups.java
index 6aff091..ada5eaf 100644
--- a/src/main/java/org/apache/aurora/scheduler/async/TaskGroups.java
+++ b/src/main/java/org/apache/aurora/scheduler/async/TaskGroups.java
@@ -18,6 +18,7 @@ package org.apache.aurora.scheduler.async;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.logging.Logger;
 
 import javax.inject.Inject;
 
@@ -29,7 +30,6 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 import com.google.common.eventbus.Subscribe;
 import com.google.common.util.concurrent.RateLimiter;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.twitter.common.application.ShutdownRegistry;
 import com.twitter.common.base.Command;
 import com.twitter.common.quantity.Amount;
@@ -39,6 +39,7 @@ import com.twitter.common.util.BackoffStrategy;
 import com.twitter.common.util.Clock;
 import com.twitter.common.util.concurrent.ExecutorServiceShutdown;
 
+import org.apache.aurora.scheduler.base.AsyncUtil;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Tasks;
 import org.apache.aurora.scheduler.events.PubsubEvent.EventSubscriber;
@@ -65,6 +66,8 @@ import static org.apache.aurora.gen.ScheduleStatus.PENDING;
  */
 public class TaskGroups implements EventSubscriber {
 
+  private static final Logger LOG = Logger.getLogger(TaskGroups.class.getName());
+
   private final ConcurrentMap<GroupKey, TaskGroup> groups = Maps.newConcurrentMap();
   private final ScheduledExecutorService executor;
   private final TaskScheduler taskScheduler;
@@ -155,11 +158,9 @@ public class TaskGroups implements EventSubscriber {
   }
 
   private static ScheduledExecutorService createThreadPool(ShutdownRegistry shutdownRegistry) {
-    // TODO(William Farner): Leverage ExceptionHandlingScheduledExecutorService:
-    // com.twitter.common.util.concurrent.ExceptionHandlingScheduledExecutorService
-    final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(
-        1,
-        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("TaskScheduler-%d").build());
+    final ScheduledThreadPoolExecutor executor =
+        AsyncUtil.singleThreadLoggingScheduledExecutor("TaskScheduler-%d", LOG);
+
     Stats.exportSize("schedule_queue_size", executor.getQueue());
     shutdownRegistry.addAction(new Command() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/main/java/org/apache/aurora/scheduler/base/AsyncUtil.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/base/AsyncUtil.java b/src/main/java/org/apache/aurora/scheduler/base/AsyncUtil.java
new file mode 100644
index 0000000..eb4c2b8
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/base/AsyncUtil.java
@@ -0,0 +1,94 @@
+/**
+ * Copyright 2014 Apache Software Foundation
+ *
+ * Licensed 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.aurora.scheduler.base;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.annotation.Nullable;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Utility class for facilitating async scheduling.
+ */
+public final class AsyncUtil {
+
+  private AsyncUtil() {
+    // Utility class.
+  }
+
+  /**
+   * Creates a {@link ScheduledThreadPoolExecutor} that logs unhandled errors.
+   *
+   * @param poolSize Thread pool size.
+   * @param nameFormat Thread naming format.
+   * @param logger Logger instance.
+   * @return instance of {@link ScheduledThreadPoolExecutor} enabled to log unhandled exceptions.
+   */
+  public static ScheduledThreadPoolExecutor loggingScheduledExecutor(
+      int poolSize,
+      String nameFormat,
+      final Logger logger) {
+
+    checkNotNull(nameFormat);
+
+    return new ScheduledThreadPoolExecutor(
+        poolSize,
+        new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build()) {
+
+      @Override
+      protected void afterExecute(Runnable runnable, @Nullable Throwable throwable) {
+        // See java.util.concurrent.ThreadPoolExecutor#afterExecute(Runnable, Throwable)
+        // for more details and an implementation example.
+        super.afterExecute(runnable, throwable);
+        if (throwable != null) {
+          logger.log(Level.SEVERE, throwable.toString(), throwable);
+        } else if (runnable instanceof Future) {
+          try {
+            Future<?> future = (Future<?>) runnable;
+            if (future.isDone()) {
+              future.get();
+            }
+          } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+          } catch (ExecutionException ee) {
+            logger.log(Level.SEVERE, ee.toString(), ee);
+          }
+        }
+      }
+    };
+  }
+
+  /**
+   * Creates a single-threaded {@link ScheduledThreadPoolExecutor} that logs unhandled errors.
+   *
+   * @param nameFormat Thread naming format.
+   * @param logger Logger instance.
+   * @return instance of {@link ScheduledThreadPoolExecutor} enabled to log unhandled exceptions.
+   */
+  public static ScheduledThreadPoolExecutor singleThreadLoggingScheduledExecutor(
+      String nameFormat,
+      Logger logger) {
+
+    return loggingScheduledExecutor(1, nameFormat, logger);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/main/java/org/apache/aurora/scheduler/sla/SlaModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/sla/SlaModule.java b/src/main/java/org/apache/aurora/scheduler/sla/SlaModule.java
index 0c6a2b8..8a0707e 100644
--- a/src/main/java/org/apache/aurora/scheduler/sla/SlaModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/sla/SlaModule.java
@@ -17,18 +17,12 @@ package org.apache.aurora.scheduler.sla;
 
 import java.lang.annotation.Retention;
 import java.lang.annotation.Target;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import javax.annotation.Nullable;
 import javax.inject.Inject;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.inject.AbstractModule;
 import com.google.inject.BindingAnnotation;
 import com.google.inject.Singleton;
@@ -39,6 +33,7 @@ import com.twitter.common.base.Command;
 import com.twitter.common.quantity.Amount;
 import com.twitter.common.quantity.Time;
 
+import org.apache.aurora.scheduler.base.AsyncUtil;
 import org.apache.aurora.scheduler.sla.MetricCalculator.MetricCalculatorSettings;
 
 import static java.lang.annotation.ElementType.FIELD;
@@ -65,35 +60,14 @@ public class SlaModule extends AbstractModule {
 
   @Override
   protected void configure() {
-    final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(
-        1,
-        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("SlaStat-%d").build()) {
-
-      @Override
-      protected void afterExecute(Runnable runnable, @Nullable Throwable throwable) {
-        super.afterExecute(runnable, throwable);
-        if (throwable != null) {
-          LOG.log(Level.SEVERE, throwable.toString(), throwable);
-        } else if (runnable instanceof Future) {
-          try {
-            Future<?> future = (Future<?>) runnable;
-            if (future.isDone()) {
-              future.get();
-            }
-          } catch (InterruptedException ie) {
-            Thread.currentThread().interrupt();
-          } catch (ExecutionException ee) {
-            LOG.log(Level.SEVERE, ee.toString(), ee);
-          }
-        }
-      }
-    };
-
     bind(MetricCalculatorSettings.class).toInstance(
         new MetricCalculatorSettings(SLA_REFRESH_RATE.get().as(Time.MILLISECONDS)));
 
     bind(MetricCalculator.class).in(Singleton.class);
-    bind(ScheduledExecutorService.class).annotatedWith(SlaExecutor.class).toInstance(executor);
+    bind(ScheduledExecutorService.class)
+        .annotatedWith(SlaExecutor.class)
+        .toInstance(AsyncUtil.singleThreadLoggingScheduledExecutor("SlaStat-%d", LOG));
+
     LifecycleModule.bindStartupAction(binder(), SlaUpdater.class);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java b/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
index 6c72339..74e7f7e 100644
--- a/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
@@ -17,16 +17,15 @@ package org.apache.aurora.scheduler.stats;
 
 import java.lang.annotation.Retention;
 import java.lang.annotation.Target;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.logging.Logger;
 
 import javax.inject.Inject;
 import javax.inject.Singleton;
 
 import com.google.common.base.Function;
 import com.google.common.collect.FluentIterable;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.inject.AbstractModule;
 import com.google.inject.BindingAnnotation;
 import com.twitter.common.application.modules.LifecycleModule;
@@ -39,6 +38,7 @@ import com.twitter.common.quantity.Time;
 
 import org.apache.aurora.gen.ResourceAggregate;
 import org.apache.aurora.scheduler.async.OfferQueue;
+import org.apache.aurora.scheduler.base.AsyncUtil;
 import org.apache.aurora.scheduler.base.Conversions;
 import org.apache.aurora.scheduler.configuration.Resources;
 import org.apache.aurora.scheduler.stats.SlotSizeCounter.MachineResource;
@@ -58,6 +58,8 @@ import static com.google.common.base.Preconditions.checkNotNull;
  */
 public class AsyncStatsModule extends AbstractModule {
 
+  private static final Logger LOG = Logger.getLogger(AsyncStatsModule.class.getName());
+
   @CmdLine(name = "async_task_stat_update_interval",
       help = "Interval on which to try to update resource consumption stats.")
   private static final Arg<Amount<Long, Time>> TASK_STAT_INTERVAL =
@@ -74,8 +76,8 @@ public class AsyncStatsModule extends AbstractModule {
 
   @Override
   protected void configure() {
-    final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
-        new ThreadFactoryBuilder().setNameFormat("AsyncStat-%d").setDaemon(true).build());
+    final ScheduledExecutorService executor =
+        AsyncUtil.singleThreadLoggingScheduledExecutor("AsyncStat-%d", LOG);
 
     bind(TaskStatCalculator.class).in(Singleton.class);
     bind(CachedCounters.class).in(Singleton.class);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
index 5512a44..c851eeb 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
@@ -21,7 +21,6 @@ import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 import java.util.Date;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.logging.Level;
@@ -45,6 +44,7 @@ import org.apache.aurora.gen.storage.RewriteTask;
 import org.apache.aurora.gen.storage.SaveAcceptedJob;
 import org.apache.aurora.gen.storage.SaveQuota;
 import org.apache.aurora.gen.storage.Snapshot;
+import org.apache.aurora.scheduler.base.AsyncUtil;
 import org.apache.aurora.scheduler.base.SchedulerException;
 import org.apache.aurora.scheduler.log.Log.Stream.InvalidPositionException;
 import org.apache.aurora.scheduler.log.Log.Stream.StreamAccessException;
@@ -149,7 +149,7 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
 
     ScheduledExecutorSchedulingService(ShutdownRegistry shutdownRegistry,
         Amount<Long, Time> shutdownGracePeriod) {
-      scheduledExecutor = Executors.newSingleThreadScheduledExecutor();
+      scheduledExecutor = AsyncUtil.singleThreadLoggingScheduledExecutor("LogStorage-%d", LOG);
       shutdownRegistry.addAction(
           new ExecutorServiceShutdown(scheduledExecutor, shutdownGracePeriod));
     }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/843d19da/src/test/java/org/apache/aurora/scheduler/base/AsyncUtilTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/base/AsyncUtilTest.java b/src/test/java/org/apache/aurora/scheduler/base/AsyncUtilTest.java
new file mode 100644
index 0000000..2e5875f
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/base/AsyncUtilTest.java
@@ -0,0 +1,87 @@
+/**
+ * Copyright 2014 Apache Software Foundation
+ *
+ * Licensed 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.aurora.scheduler.base;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import com.twitter.common.testing.easymock.EasyMockTest;
+
+import org.easymock.EasyMock;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.easymock.EasyMock.contains;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expectLastCall;
+
+public class AsyncUtilTest extends EasyMockTest {
+  private Logger logger;
+  private ScheduledThreadPoolExecutor executor;
+
+  @Before
+  public void setUp() {
+    logger = createMock(Logger.class);
+    executor = AsyncUtil.singleThreadLoggingScheduledExecutor("Test-%d", logger);
+  }
+
+  @Test(expected = ExecutionException.class)
+  public void testScheduleLogging() throws Exception {
+    logger.log(
+        eq(Level.SEVERE),
+        contains("Expected exception."),
+        EasyMock.<ExecutionException>anyObject());
+
+    expectLastCall().times(1);
+
+    control.replay();
+
+    ScheduledFuture<?> future = executor.schedule(new Runnable() {
+      @Override
+      public void run() {
+        throw new IllegalArgumentException("Expected exception.");
+      }
+    }, 0, TimeUnit.MILLISECONDS);
+
+    future.get();
+  }
+
+  @Test(expected = ExecutionException.class)
+  public void testSubmitLogging() throws Exception {
+    logger.log(
+        eq(Level.SEVERE),
+        contains("Expected exception."),
+        EasyMock.<ExecutionException>anyObject());
+
+    expectLastCall().times(1);
+
+    control.replay();
+
+    Future<?> future = executor.submit(new Runnable() {
+      @Override
+      public void run() {
+        throw new IllegalArgumentException("Expected exception.");
+      }
+    });
+
+    future.get();
+  }
+}