You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by do...@apache.org on 2018/05/01 01:14:21 UTC

[15/16] reef git commit: [REEF-2012] Add driver restart capabilities to reef runtime mock

[REEF-2012] Add driver restart capabilities to reef runtime mock

JIRA: [REEF-2012](https://issues.apache.org/jira/browse/REEF-2012)

Pull Request:
   Closes #1452


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

Branch: refs/heads/REEF-335
Commit: 5ed56eba3b0b438dc0fdf10a37a58fcfdeb81223
Parents: 35df820
Author: Tyson Condie <tc...@apache.org>
Authored: Thu Apr 26 12:52:51 2018 -0700
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Mon Apr 30 11:09:13 2018 -0700

----------------------------------------------------------------------
 .../org/apache/reef/mock/AutoCompletable.java   |  45 --
 .../mock/DefaultTaskReturnValueProvider.java    |  41 --
 .../org/apache/reef/mock/MockConfiguration.java | 173 ------
 .../java/org/apache/reef/mock/MockFailure.java  |  67 ---
 .../java/org/apache/reef/mock/MockRuntime.java  |  76 ---
 .../reef/mock/MockTaskReturnValueProvider.java  |  44 --
 .../org/apache/reef/mock/ProcessRequest.java    |  54 --
 .../reef/mock/driver/AutoCompletable.java       |  45 ++
 .../driver/DefaultTaskReturnValueProvider.java  |  41 ++
 .../reef/mock/driver/MockConfiguration.java     | 208 ++++++++
 .../mock/driver/MockDriverRestartContext.java   | 172 ++++++
 .../apache/reef/mock/driver/MockFailure.java    |  74 +++
 .../apache/reef/mock/driver/MockRuntime.java    |  92 ++++
 .../driver/MockTaskReturnValueProvider.java     |  44 ++
 .../apache/reef/mock/driver/ProcessRequest.java |  54 ++
 .../apache/reef/mock/driver/package-info.java   |  40 ++
 .../mock/driver/request/AllocateEvaluator.java  |  72 +++
 .../reef/mock/driver/request/CloseContext.java  |  76 +++
 .../mock/driver/request/CloseEvaluator.java     |  78 +++
 .../reef/mock/driver/request/CloseTask.java     |  89 ++++
 .../reef/mock/driver/request/CompleteTask.java  |  82 +++
 .../reef/mock/driver/request/CreateContext.java |  76 +++
 .../driver/request/CreateContextAndTask.java    |  98 ++++
 .../reef/mock/driver/request/CreateTask.java    |  89 ++++
 .../driver/request/ProcessRequestInternal.java  |  44 ++
 .../request/SendMessageDriverToContext.java     |  81 +++
 .../driver/request/SendMessageDriverToTask.java |  81 +++
 .../reef/mock/driver/request/SuspendTask.java   |  90 ++++
 .../reef/mock/driver/request/package-info.java  |  23 +
 .../mock/driver/runtime/MockActiveContext.java  | 139 +++++
 .../driver/runtime/MockAllocatedEvaluator.java  | 172 ++++++
 .../reef/mock/driver/runtime/MockClock.java     | 120 +++++
 .../mock/driver/runtime/MockClosedContext.java  |  71 +++
 .../mock/driver/runtime/MockCompletedTask.java  |  57 ++
 .../driver/runtime/MockEvaluatorDescriptor.java |  64 +++
 .../driver/runtime/MockEvaluatorRequestor.java  |  85 +++
 .../mock/driver/runtime/MockFailedContext.java  |  93 ++++
 .../driver/runtime/MockFailedEvaluator.java     |  79 +++
 .../mock/driver/runtime/MockNodeDescriptor.java |  68 +++
 .../mock/driver/runtime/MockRunningTask.java    |  97 ++++
 .../mock/driver/runtime/MockRuntimeDriver.java  | 522 +++++++++++++++++++
 .../mock/driver/runtime/MockSuspendedTask.java  |  54 ++
 .../reef/mock/driver/runtime/MockUtils.java     |  48 ++
 .../reef/mock/driver/runtime/package-info.java  |  23 +
 .../java/org/apache/reef/mock/package-info.java |  40 --
 .../reef/mock/request/AllocateEvaluator.java    |  72 ---
 .../apache/reef/mock/request/CloseContext.java  |  76 ---
 .../reef/mock/request/CloseEvaluator.java       |  78 ---
 .../org/apache/reef/mock/request/CloseTask.java |  89 ----
 .../apache/reef/mock/request/CompleteTask.java  |  82 ---
 .../apache/reef/mock/request/CreateContext.java |  76 ---
 .../reef/mock/request/CreateContextAndTask.java |  98 ----
 .../apache/reef/mock/request/CreateTask.java    |  89 ----
 .../mock/request/ProcessRequestInternal.java    |  44 --
 .../request/SendMessageDriverToContext.java     |  81 ---
 .../mock/request/SendMessageDriverToTask.java   |  81 ---
 .../apache/reef/mock/request/SuspendTask.java   |  90 ----
 .../apache/reef/mock/request/package-info.java  |  23 -
 .../reef/mock/runtime/MockActiveContext.java    | 139 -----
 .../mock/runtime/MockAllocatedEvalautor.java    | 149 ------
 .../org/apache/reef/mock/runtime/MockClock.java | 120 -----
 .../reef/mock/runtime/MockClosedContext.java    |  71 ---
 .../reef/mock/runtime/MockCompletedTask.java    |  57 --
 .../mock/runtime/MockEvaluatorDescriptor.java   |  64 ---
 .../mock/runtime/MockEvaluatorRequestor.java    |  85 ---
 .../reef/mock/runtime/MockFailedContext.java    |  93 ----
 .../reef/mock/runtime/MockFailedEvaluator.java  |  79 ---
 .../reef/mock/runtime/MockNodeDescriptor.java   |  68 ---
 .../reef/mock/runtime/MockRunningTask.java      |  97 ----
 .../reef/mock/runtime/MockRuntimeDriver.java    | 454 ----------------
 .../reef/mock/runtime/MockSuspendedTask.java    |  54 --
 .../org/apache/reef/mock/runtime/MockUtils.java |  48 --
 .../apache/reef/mock/runtime/package-info.java  |  23 -
 .../org/apache/reef/mock/BasicMockTests.java    | 205 --------
 .../org/apache/reef/mock/MockApplication.java   | 275 ----------
 .../apache/reef/mock/driver/BasicMockTests.java | 163 ++++++
 .../reef/mock/driver/MockApplication.java       | 275 ++++++++++
 .../apache/reef/mock/driver/package-info.java   |  23 +
 .../java/org/apache/reef/mock/package-info.java |  23 -
 .../driver/YarnDriverRestartConfiguration.java  |   2 +-
 .../driver/YarnDriverRuntimeRestartManager.java |   2 +-
 81 files changed, 3904 insertions(+), 3625 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/AutoCompletable.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/AutoCompletable.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/AutoCompletable.java
deleted file mode 100644
index 173b410..0000000
--- a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/AutoCompletable.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.mock;
-
-import org.apache.reef.annotations.Unstable;
-
-/**
- * Indicates that a process request should auto complete.
- */
-@Unstable
-public interface AutoCompletable {
-
-  /**
-   * @return true if should auto complete
-   */
-  boolean doAutoComplete();
-
-  /**
-   * Set auto complete.
-   * @param value to set
-   */
-  void setAutoComplete(final boolean value);
-
-  /**
-   * @return auto complete process request
-   */
-  ProcessRequest getCompletionProcessRequest();
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/DefaultTaskReturnValueProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/DefaultTaskReturnValueProvider.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/DefaultTaskReturnValueProvider.java
deleted file mode 100644
index 2ae81d4..0000000
--- a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/DefaultTaskReturnValueProvider.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing,
- *  software distributed under the License is distributed on an
- *  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- *  KIND, either express or implied.  See the License for the
- *  specific language governing permissions and limitations
- *  under the License.
- *
- */
-
-package org.apache.reef.mock;
-
-import org.apache.reef.mock.runtime.MockRunningTask;
-
-import javax.inject.Inject;
-
-/**
- * A default task return value provider.
- */
-final class DefaultTaskReturnValueProvider implements MockTaskReturnValueProvider {
-
-  @Inject
-  DefaultTaskReturnValueProvider() {
-
-  }
-
-  @Override
-  public byte[] getReturnValue(final MockRunningTask task) {
-    return new byte[0];
-  }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockConfiguration.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockConfiguration.java
deleted file mode 100644
index beec5c4..0000000
--- a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockConfiguration.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.reef.mock;
-
-import org.apache.reef.annotations.Unstable;
-import org.apache.reef.driver.client.JobMessageObserver;
-import org.apache.reef.driver.context.ActiveContext;
-import org.apache.reef.driver.context.ClosedContext;
-import org.apache.reef.driver.context.ContextMessage;
-import org.apache.reef.driver.context.FailedContext;
-import org.apache.reef.driver.evaluator.AllocatedEvaluator;
-import org.apache.reef.driver.evaluator.CompletedEvaluator;
-import org.apache.reef.driver.evaluator.EvaluatorRequestor;
-import org.apache.reef.driver.evaluator.FailedEvaluator;
-import org.apache.reef.driver.parameters.*;
-import org.apache.reef.driver.task.*;
-import org.apache.reef.mock.runtime.MockClock;
-import org.apache.reef.mock.runtime.MockEvaluatorRequestor;
-import org.apache.reef.mock.runtime.MockRuntimeDriver;
-import org.apache.reef.tang.formats.ConfigurationModule;
-import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
-import org.apache.reef.tang.formats.OptionalImpl;
-import org.apache.reef.tang.formats.RequiredImpl;
-import org.apache.reef.wake.EventHandler;
-import org.apache.reef.wake.time.Clock;
-import org.apache.reef.wake.time.event.StartTime;
-import org.apache.reef.wake.time.event.StopTime;
-
-/**
- * Configure a mock runtime.
- */
-@Unstable
-public class MockConfiguration extends ConfigurationModuleBuilder {
-
-  /**
-   * The event handler invoked right after the driver boots up.
-   */
-  public static final RequiredImpl<EventHandler<StartTime>> ON_DRIVER_STARTED = new RequiredImpl<>();
-
-  /**
-   * The event handler invoked right before the driver shuts down. Defaults to ignore.
-   */
-  public static final OptionalImpl<EventHandler<StopTime>> ON_DRIVER_STOP = new OptionalImpl<>();
-
-  // ***** EVALUATOR HANDLER BINDINGS:
-
-  /**
-   * Event handler for allocated evaluators. Defaults to returning the evaluator if not bound.
-   */
-  public static final OptionalImpl<EventHandler<AllocatedEvaluator>> ON_EVALUATOR_ALLOCATED = new OptionalImpl<>();
-
-  /**
-   * Event handler for completed evaluators. Defaults to logging if not bound.
-   */
-  public static final OptionalImpl<EventHandler<CompletedEvaluator>> ON_EVALUATOR_COMPLETED = new OptionalImpl<>();
-
-  /**
-   * Event handler for failed evaluators. Defaults to job failure if not bound.
-   */
-  public static final OptionalImpl<EventHandler<FailedEvaluator>> ON_EVALUATOR_FAILED = new OptionalImpl<>();
-
-  // ***** TASK HANDLER BINDINGS:
-
-  /**
-   * Event handler for task messages. Defaults to logging if not bound.
-   */
-  public static final OptionalImpl<EventHandler<TaskMessage>> ON_TASK_MESSAGE = new OptionalImpl<>();
-
-  /**
-   * Event handler for completed tasks. Defaults to closing the context the task ran on if not bound.
-   */
-  public static final OptionalImpl<EventHandler<CompletedTask>> ON_TASK_COMPLETED = new OptionalImpl<>();
-
-  /**
-   * Event handler for failed tasks. Defaults to job failure if not bound.
-   */
-  public static final OptionalImpl<EventHandler<FailedTask>> ON_TASK_FAILED = new OptionalImpl<>();
-
-  /**
-   * Event handler for running tasks. Defaults to logging if not bound.
-   */
-  public static final OptionalImpl<EventHandler<RunningTask>> ON_TASK_RUNNING = new OptionalImpl<>();
-
-  /**
-   * Event handler for suspended tasks. Defaults to job failure if not bound. Rationale: many jobs don't support
-   * task suspension. Hence, this parameter should be optional. The only sane default is to crash the job, then.
-   */
-  public static final OptionalImpl<EventHandler<SuspendedTask>> ON_TASK_SUSPENDED = new OptionalImpl<>();
-
-  // ***** CONTEXT HANDLER BINDINGS:
-
-  /**
-   * Event handler for active context. Defaults to closing the context if not bound.
-   */
-  public static final OptionalImpl<EventHandler<ActiveContext>> ON_CONTEXT_ACTIVE = new OptionalImpl<>();
-
-  /**
-   * Event handler for closed context. Defaults to logging if not bound.
-   */
-  public static final OptionalImpl<EventHandler<ClosedContext>> ON_CONTEXT_CLOSED = new OptionalImpl<>();
-
-  /**
-   * Event handler for closed context. Defaults to job failure if not bound.
-   */
-  public static final OptionalImpl<EventHandler<FailedContext>> ON_CONTEXT_FAILED = new OptionalImpl<>();
-
-  /**
-   * Event handler for context messages. Defaults to logging if not bound.
-   */
-  public static final OptionalImpl<EventHandler<ContextMessage>> ON_CONTEXT_MESSAGE = new OptionalImpl<>();
-
-
-  /**
-   * Receiver of messages sent by the Driver to the client.
-   */
-  public static final OptionalImpl<JobMessageObserver> ON_JOB_MESSAGE = new OptionalImpl<>();
-
-  /**
-   *  An implementation of a task return value provider.
-   */
-  public static final OptionalImpl<MockTaskReturnValueProvider> TASK_RETURN_VALUE_PROVIDER = new OptionalImpl<>();
-
-  public static final ConfigurationModule CONF = new MockConfiguration()
-      .bindImplementation(EvaluatorRequestor.class, MockEvaluatorRequestor.class) // requesting evaluators
-      .bindImplementation(MockRuntime.class, MockRuntimeDriver.class)
-      .bindImplementation(MockFailure.class, MockRuntimeDriver.class)
-      .bindImplementation(Clock.class, MockClock.class)
-      .bindImplementation(MockTaskReturnValueProvider.class, TASK_RETURN_VALUE_PROVIDER)
-
-      // client handlers
-      .bindImplementation(JobMessageObserver.class, ON_JOB_MESSAGE) // sending message to job client
-
-      // Driver start/stop handlers
-      .bindSetEntry(DriverStartHandler.class, ON_DRIVER_STARTED)
-      .bindSetEntry(Clock.StopHandler.class, ON_DRIVER_STOP)
-
-      // Evaluator handlers
-      .bindSetEntry(EvaluatorAllocatedHandlers.class, ON_EVALUATOR_ALLOCATED)
-      .bindSetEntry(EvaluatorCompletedHandlers.class, ON_EVALUATOR_COMPLETED)
-      .bindSetEntry(EvaluatorFailedHandlers.class, ON_EVALUATOR_FAILED)
-
-      // Task handlers
-      .bindSetEntry(TaskRunningHandlers.class, ON_TASK_RUNNING)
-      .bindSetEntry(TaskFailedHandlers.class, ON_TASK_FAILED)
-      .bindSetEntry(TaskMessageHandlers.class, ON_TASK_MESSAGE)
-      .bindSetEntry(TaskCompletedHandlers.class, ON_TASK_COMPLETED)
-      .bindSetEntry(TaskSuspendedHandlers.class, ON_TASK_SUSPENDED)
-
-      // Context handlers
-      .bindSetEntry(ContextActiveHandlers.class, ON_CONTEXT_ACTIVE)
-      .bindSetEntry(ContextClosedHandlers.class, ON_CONTEXT_CLOSED)
-      .bindSetEntry(ContextMessageHandlers.class, ON_CONTEXT_MESSAGE)
-      .bindSetEntry(ContextFailedHandlers.class, ON_CONTEXT_FAILED)
-
-      .build();
-
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockFailure.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockFailure.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockFailure.java
deleted file mode 100644
index f8822a2..0000000
--- a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockFailure.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing,
- *  software distributed under the License is distributed on an
- *  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- *  KIND, either express or implied.  See the License for the
- *  specific language governing permissions and limitations
- *  under the License.
- *
- */
-package org.apache.reef.mock;
-
-import org.apache.reef.annotations.Unstable;
-import org.apache.reef.driver.context.ActiveContext;
-import org.apache.reef.driver.evaluator.AllocatedEvaluator;
-import org.apache.reef.driver.task.RunningTask;
-
-import java.util.Collection;
-
-/**
- * Used to fail running REEF entities i.e., Evaluators, Contexts, Tasks.
- */
-@Unstable
-public interface MockFailure {
-
-  /**
-   * @return current Collection of allocated evaluators.
-   */
-  Collection<AllocatedEvaluator> getCurrentAllocatedEvaluators();
-
-  /**
-   * Fail an allocated evaluator.
-   * @param evaluator to be failed
-   */
-  void fail(final AllocatedEvaluator evaluator);
-
-  /**
-   * @return current Collection of active contexts
-   */
-  Collection<ActiveContext> getCurrentActiveContexts();
-
-  /**
-   * Fail an ActiveContext.
-   * @param context to be failed
-   */
-  void fail(final ActiveContext context);
-
-  /**
-   * @return current Collection of running tasks
-   */
-  Collection<RunningTask> getCurrentRunningTasks();
-
-  /**
-   * Fail a running task.
-   * @param task to be failed
-   */
-  void fail(final RunningTask task);
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockRuntime.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockRuntime.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockRuntime.java
deleted file mode 100644
index 0e09f5d..0000000
--- a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockRuntime.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.mock;
-
-import org.apache.reef.annotations.Unstable;
-
-/**
- * Mock API used to drive the evaluation of ProcessRequest
- * events, which are triggered by the Application driver.
- * Clients used this to determine whether a particular ProcessRequest
- * event should succeed or fail.
- */
-@Unstable
-public interface MockRuntime extends MockFailure {
-
-  /**
-   * Initiate the start time event to the application driver.
-   */
-  void start();
-
-  /**
-   * Initiate the stop time event to the application driver.
-   */
-  void stop();
-
-  /**
-   * @return true if there is an outstanding ProcessRequest
-   */
-  boolean hasProcessRequest();
-
-  /**
-   * The client (caller) is responsible for determining what
-   * to do with a ProcessRequest event. There are three options:
-   * 1. Pass to the succeed method, which signals success to the driver.
-   * 2. Pass to the fail method, signaling failure to the driver.
-   * 3. Drop it on the floor (e.g., network failure).
-   *
-   * @return the next ProcessRequest object to be processed.
-   */
-  ProcessRequest getNextProcessRequest();
-
-  /**
-   * The driver will be informed that the operation corresponding
-   * to the ProcessRequest succeeded, and will be given any relevant
-   * data structures e.g., AllocatedEvaluator, RunningTask, etc.
-   *
-   * @param request to be processed successfully
-   */
-  void succeed(final ProcessRequest request);
-
-  /**
-   * The driver will be informed that the operation corresponding
-   * to the PRocessRequest failed, and will be given any relevant
-   * data structures e.g., FailedEvaluator, FailedTask, etc.
-   *
-   * @param request to be failed.
-   */
-  void fail(final ProcessRequest request);
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockTaskReturnValueProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockTaskReturnValueProvider.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockTaskReturnValueProvider.java
deleted file mode 100644
index a0e794b..0000000
--- a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/MockTaskReturnValueProvider.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing,
- *  software distributed under the License is distributed on an
- *  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- *  KIND, either express or implied.  See the License for the
- *  specific language governing permissions and limitations
- *  under the License.
- *
- */
-
-package org.apache.reef.mock;
-
-import org.apache.reef.annotations.Unstable;
-import org.apache.reef.driver.task.CompletedTask;
-import org.apache.reef.mock.runtime.MockRunningTask;
-import org.apache.reef.tang.annotations.DefaultImplementation;
-
-/**
- * Clients bind an implementation of this interface, which
- * will be used to create a mock return value for a mock
- * task execution. This return value will be returned by
- * the {@link CompletedTask#get()}} method.
- */
-@Unstable
-@DefaultImplementation(DefaultTaskReturnValueProvider.class)
-public interface MockTaskReturnValueProvider {
-
-  /**
-   * Provide a valid return value for the {@link CompletedTask#get()} method.
-   * @param task that is to be provided with a return value
-   * @return {@link org.apache.reef.task.Task#call(byte[])} return value
-   */
-  byte[] getReturnValue(final MockRunningTask task);
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/ProcessRequest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/ProcessRequest.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/ProcessRequest.java
deleted file mode 100644
index 09e9691..0000000
--- a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/ProcessRequest.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.mock;
-
-import org.apache.reef.annotations.Unstable;
-
-/**
- * A ProcessRequest refers to an outstanding event that is
- * waiting to be processed by the REEF mock runtime. Clients
- * are responsible for deciding how a ProcessRequest should be
- * handled, by either:
- * 1. successfully processing the request
- * 2. unsucessfully processing the request
- * 3. dropping the processing request (i.e., loosing it)
- * These decisions are conveyed through the {MockRuntime} API.
- */
-@Unstable
-public interface ProcessRequest extends AutoCompletable {
-  /**
-   * process request type.
-   */
-  enum Type {
-    ALLOCATE_EVALUATOR,
-    CLOSE_EVALUATOR,
-    CREATE_CONTEXT,
-    CLOSE_CONTEXT,
-    CREATE_TASK,
-    SUSPEND_TASK,
-    CLOSE_TASK,
-    COMPLETE_TASK,
-    CREATE_CONTEXT_AND_TASK,
-    SEND_MESSAGE_DRIVER_TO_TASK,
-    SEND_MESSAGE_DRIVER_TO_CONTEXT
-  }
-
-  Type getType();
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/AutoCompletable.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/AutoCompletable.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/AutoCompletable.java
new file mode 100644
index 0000000..3ecb0b3
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/AutoCompletable.java
@@ -0,0 +1,45 @@
+/*
+ * 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.reef.mock.driver;
+
+import org.apache.reef.annotations.Unstable;
+
+/**
+ * Indicates that a process request should auto complete.
+ */
+@Unstable
+public interface AutoCompletable {
+
+  /**
+   * @return true if should auto complete
+   */
+  boolean doAutoComplete();
+
+  /**
+   * Set auto complete.
+   * @param value to set
+   */
+  void setAutoComplete(final boolean value);
+
+  /**
+   * @return auto complete process request
+   */
+  ProcessRequest getCompletionProcessRequest();
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/DefaultTaskReturnValueProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/DefaultTaskReturnValueProvider.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/DefaultTaskReturnValueProvider.java
new file mode 100644
index 0000000..d3a6be7
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/DefaultTaskReturnValueProvider.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing,
+ *  software distributed under the License is distributed on an
+ *  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ *  specific language governing permissions and limitations
+ *  under the License.
+ *
+ */
+
+package org.apache.reef.mock.driver;
+
+import org.apache.reef.mock.driver.runtime.MockRunningTask;
+
+import javax.inject.Inject;
+
+/**
+ * A default task return value provider.
+ */
+final class DefaultTaskReturnValueProvider implements MockTaskReturnValueProvider {
+
+  @Inject
+  DefaultTaskReturnValueProvider() {
+
+  }
+
+  @Override
+  public byte[] getReturnValue(final MockRunningTask task) {
+    return new byte[0];
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockConfiguration.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockConfiguration.java
new file mode 100644
index 0000000..3366220
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockConfiguration.java
@@ -0,0 +1,208 @@
+/*
+ * 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.reef.mock.driver;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.driver.client.JobMessageObserver;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.context.ClosedContext;
+import org.apache.reef.driver.context.ContextMessage;
+import org.apache.reef.driver.context.FailedContext;
+import org.apache.reef.driver.evaluator.AllocatedEvaluator;
+import org.apache.reef.driver.evaluator.CompletedEvaluator;
+import org.apache.reef.driver.evaluator.EvaluatorRequestor;
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.driver.parameters.*;
+import org.apache.reef.driver.restart.DriverRestartCompleted;
+import org.apache.reef.driver.restart.DriverRestarted;
+import org.apache.reef.driver.task.*;
+import org.apache.reef.mock.driver.runtime.MockClock;
+import org.apache.reef.mock.driver.runtime.MockEvaluatorRequestor;
+import org.apache.reef.mock.driver.runtime.MockRuntimeDriver;
+import org.apache.reef.tang.formats.ConfigurationModule;
+import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
+import org.apache.reef.tang.formats.OptionalImpl;
+import org.apache.reef.tang.formats.RequiredImpl;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.Clock;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+
+/**
+ * Configure a mock runtime.
+ */
+@Unstable
+public class MockConfiguration extends ConfigurationModuleBuilder {
+
+  /**
+   * The event handler invoked right after the driver boots up.
+   */
+  public static final RequiredImpl<EventHandler<StartTime>> ON_DRIVER_STARTED = new RequiredImpl<>();
+
+  /**
+   * The event handler invoked right before the driver shuts down. Defaults to ignore.
+   */
+  public static final OptionalImpl<EventHandler<StopTime>> ON_DRIVER_STOP = new OptionalImpl<>();
+
+  // ***** EVALUATOR HANDLER BINDINGS:
+
+  /**
+   * Event handler for allocated evaluators. Defaults to returning the evaluator if not bound.
+   */
+  public static final OptionalImpl<EventHandler<AllocatedEvaluator>> ON_EVALUATOR_ALLOCATED = new OptionalImpl<>();
+
+  /**
+   * Event handler for completed evaluators. Defaults to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<CompletedEvaluator>> ON_EVALUATOR_COMPLETED = new OptionalImpl<>();
+
+  /**
+   * Event handler for failed evaluators. Defaults to job failure if not bound.
+   */
+  public static final OptionalImpl<EventHandler<FailedEvaluator>> ON_EVALUATOR_FAILED = new OptionalImpl<>();
+
+  // ***** TASK HANDLER BINDINGS:
+
+  /**
+   * Event handler for task messages. Defaults to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<TaskMessage>> ON_TASK_MESSAGE = new OptionalImpl<>();
+
+  /**
+   * Event handler for completed tasks. Defaults to closing the context the task ran on if not bound.
+   */
+  public static final OptionalImpl<EventHandler<CompletedTask>> ON_TASK_COMPLETED = new OptionalImpl<>();
+
+  /**
+   * Event handler for failed tasks. Defaults to job failure if not bound.
+   */
+  public static final OptionalImpl<EventHandler<FailedTask>> ON_TASK_FAILED = new OptionalImpl<>();
+
+  /**
+   * Event handler for running tasks. Defaults to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<RunningTask>> ON_TASK_RUNNING = new OptionalImpl<>();
+
+  /**
+   * Event handler for suspended tasks. Defaults to job failure if not bound. Rationale: many jobs don't support
+   * task suspension. Hence, this parameter should be optional. The only sane default is to crash the job, then.
+   */
+  public static final OptionalImpl<EventHandler<SuspendedTask>> ON_TASK_SUSPENDED = new OptionalImpl<>();
+
+  // ***** CONTEXT HANDLER BINDINGS:
+
+  /**
+   * Event handler for active context. Defaults to closing the context if not bound.
+   */
+  public static final OptionalImpl<EventHandler<ActiveContext>> ON_CONTEXT_ACTIVE = new OptionalImpl<>();
+
+  /**
+   * Event handler for closed context. Defaults to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<ClosedContext>> ON_CONTEXT_CLOSED = new OptionalImpl<>();
+
+  /**
+   * Event handler for closed context. Defaults to job failure if not bound.
+   */
+  public static final OptionalImpl<EventHandler<FailedContext>> ON_CONTEXT_FAILED = new OptionalImpl<>();
+
+  /**
+   * Event handler for context messages. Defaults to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<ContextMessage>> ON_CONTEXT_MESSAGE = new OptionalImpl<>();
+
+  /**
+   * This event is fired in place of the ON_DRIVER_STARTED when the Driver is in fact restarted after failure.
+   */
+  public static final OptionalImpl<EventHandler<DriverRestarted>> ON_DRIVER_RESTARTED = new OptionalImpl<>();
+
+  /**
+   * Event handler for running tasks in previous evaluator, when driver restarted. Defaults to crash if not bound.
+   */
+  public static final OptionalImpl<EventHandler<RunningTask>> ON_DRIVER_RESTART_TASK_RUNNING = new OptionalImpl<>();
+
+  /**
+   * Event handler for active context when driver restart. Defaults to closing the context if not bound.
+   */
+  public static final OptionalImpl<EventHandler<ActiveContext>> ON_DRIVER_RESTART_CONTEXT_ACTIVE = new OptionalImpl<>();
+
+  /**
+   * Event handler for the event of driver restart completion, default to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<DriverRestartCompleted>> ON_DRIVER_RESTART_COMPLETED =
+      new OptionalImpl<>();
+
+  /**
+   * Event handler for the event of driver restart completion, default to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<FailedEvaluator>> ON_DRIVER_RESTART_EVALUATOR_FAILED =
+      new OptionalImpl<>();
+
+  /**
+   * Receiver of messages sent by the Driver to the client.
+   */
+  public static final OptionalImpl<JobMessageObserver> ON_JOB_MESSAGE = new OptionalImpl<>();
+
+  /**
+   *  An implementation of a task return value provider.
+   */
+  public static final OptionalImpl<MockTaskReturnValueProvider> TASK_RETURN_VALUE_PROVIDER = new OptionalImpl<>();
+
+  public static final ConfigurationModule CONF = new MockConfiguration()
+      .bindImplementation(EvaluatorRequestor.class, MockEvaluatorRequestor.class) // requesting evaluators
+      .bindImplementation(MockRuntime.class, MockRuntimeDriver.class)
+      .bindImplementation(MockFailure.class, MockRuntimeDriver.class)
+      .bindImplementation(Clock.class, MockClock.class)
+      .bindImplementation(MockTaskReturnValueProvider.class, TASK_RETURN_VALUE_PROVIDER)
+
+      // recovery handlers
+      .bindSetEntry(DriverRestartFailedEvaluatorHandlers.class, ON_DRIVER_RESTART_EVALUATOR_FAILED)
+      .bindSetEntry(DriverRestartCompletedHandlers.class, ON_DRIVER_RESTART_COMPLETED)
+      .bindSetEntry(DriverRestartContextActiveHandlers.class, ON_DRIVER_RESTART_CONTEXT_ACTIVE)
+      .bindSetEntry(DriverRestartTaskRunningHandlers.class, ON_DRIVER_RESTART_TASK_RUNNING)
+      .bindSetEntry(DriverRestartHandler.class, ON_DRIVER_RESTARTED)
+
+      // client handlers
+      .bindImplementation(JobMessageObserver.class, ON_JOB_MESSAGE) // sending message to job client
+
+      // Driver start/stop handlers
+      .bindSetEntry(DriverStartHandler.class, ON_DRIVER_STARTED)
+      .bindSetEntry(Clock.StopHandler.class, ON_DRIVER_STOP)
+
+      // Evaluator handlers
+      .bindSetEntry(EvaluatorAllocatedHandlers.class, ON_EVALUATOR_ALLOCATED)
+      .bindSetEntry(EvaluatorCompletedHandlers.class, ON_EVALUATOR_COMPLETED)
+      .bindSetEntry(EvaluatorFailedHandlers.class, ON_EVALUATOR_FAILED)
+
+      // Task handlers
+      .bindSetEntry(TaskRunningHandlers.class, ON_TASK_RUNNING)
+      .bindSetEntry(TaskFailedHandlers.class, ON_TASK_FAILED)
+      .bindSetEntry(TaskMessageHandlers.class, ON_TASK_MESSAGE)
+      .bindSetEntry(TaskCompletedHandlers.class, ON_TASK_COMPLETED)
+      .bindSetEntry(TaskSuspendedHandlers.class, ON_TASK_SUSPENDED)
+
+      // Context handlers
+      .bindSetEntry(ContextActiveHandlers.class, ON_CONTEXT_ACTIVE)
+      .bindSetEntry(ContextClosedHandlers.class, ON_CONTEXT_CLOSED)
+      .bindSetEntry(ContextMessageHandlers.class, ON_CONTEXT_MESSAGE)
+      .bindSetEntry(ContextFailedHandlers.class, ON_CONTEXT_FAILED)
+
+      .build();
+
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockDriverRestartContext.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockDriverRestartContext.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockDriverRestartContext.java
new file mode 100644
index 0000000..4c0ec28
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockDriverRestartContext.java
@@ -0,0 +1,172 @@
+/*
+ * 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.reef.mock.driver;
+
+import org.apache.reef.driver.restart.DriverRestartCompleted;
+import org.apache.reef.driver.restart.DriverRestarted;
+import org.apache.reef.mock.driver.runtime.MockActiveContext;
+import org.apache.reef.mock.driver.runtime.MockAllocatedEvaluator;
+import org.apache.reef.mock.driver.runtime.MockFailedEvaluator;
+import org.apache.reef.mock.driver.runtime.MockRunningTask;
+import org.apache.reef.wake.time.Time;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+
+import java.util.*;
+
+/**
+ * Contains the runtime driver state at the time of a driver
+ * failure, triggered by {@link MockFailure}.
+ */
+public final class MockDriverRestartContext {
+
+  private final int restartAttemps;
+
+  private final StartTime startTime;
+
+  private final List<MockAllocatedEvaluator> allocatedEvaluators;
+
+  private final List<MockActiveContext> activeContexts;
+
+  private final List<MockRunningTask> runningTasks;
+
+  private final List<MockFailedEvaluator> failedEvaluators;
+
+  public MockDriverRestartContext(
+      final int restartAttemps,
+      final StartTime startTime,
+      final List<MockAllocatedEvaluator> allocatedEvaluators,
+      final List<MockActiveContext> activeContexts,
+      final List<MockRunningTask> runningTasks) {
+    this.restartAttemps = restartAttemps;
+    this.startTime = startTime;
+    this.allocatedEvaluators = allocatedEvaluators;
+    this.activeContexts = activeContexts;
+    this.runningTasks = runningTasks;
+    this.failedEvaluators = new ArrayList<>();
+  }
+
+  /**
+   * Generate a DriverRestarted event to be passed to the
+   * {@link org.apache.reef.driver.parameters.DriverRestartHandler}.
+   * @return DriverRestarted event based on the state at the time of driver failure
+   */
+  public DriverRestarted getDriverRestarted() {
+    final Set<String> expectedEvaluatorIds = new HashSet<>();
+    for (final MockAllocatedEvaluator allocatedEvaluator : this.allocatedEvaluators) {
+      expectedEvaluatorIds.add(allocatedEvaluator.getId());
+    }
+    return new DriverRestarted() {
+      @Override
+      public int getResubmissionAttempts() {
+        return restartAttemps;
+      }
+
+      @Override
+      public StartTime getStartTime() {
+        return startTime;
+      }
+
+      @Override
+      public Set<String> getExpectedEvaluatorIds() {
+        return expectedEvaluatorIds;
+      }
+    };
+  }
+
+  public DriverRestartCompleted getDriverRestartCompleted(final boolean isTimeout, final long restartDuration) {
+    return new DriverRestartCompleted() {
+      @Override
+      public Time getCompletedTime() {
+        return new StopTime(startTime.getTimestamp() + restartDuration);
+      }
+
+      @Override
+      public boolean isTimedOut() {
+        return isTimeout;
+      }
+    };
+  }
+
+  /**
+   * Pass these tasks to the {@link org.apache.reef.driver.parameters.DriverRestartTaskRunningHandlers}.
+   * @return MockRunningTasks at the time of driver failure
+   */
+  public List<MockRunningTask> getRunningTasks() {
+    return this.runningTasks;
+  }
+
+  /**
+   * Pass these active contexts to the {@link org.apache.reef.driver.parameters.DriverRestartContextActiveHandlers}.
+   * These active contexts have no tasks running.
+   * @return
+   */
+  public List<MockActiveContext> getIdleActiveContexts() {
+    final List<MockActiveContext> idleActiveContexts = new ArrayList<>();
+    final Set<String> activeContextsWithRunningTasks = new HashSet<>();
+    for (final MockRunningTask task : this.runningTasks) {
+      activeContextsWithRunningTasks.add(task.getActiveContext().getEvaluatorId());
+    }
+    for (final MockActiveContext context : this.activeContexts) {
+      if (!activeContextsWithRunningTasks.contains(context.getEvaluatorId())) {
+        idleActiveContexts.add(context);
+      }
+    }
+    return idleActiveContexts;
+  }
+
+  public List<MockFailedEvaluator> getFailedEvaluators() {
+    return this.failedEvaluators;
+  }
+
+  /**
+   * Fail a task.
+   * @param task to fail
+   */
+  public void failTask(final MockRunningTask task) {
+    this.runningTasks.remove(task);
+  }
+
+  /**
+   * Fail an evaluator; automatically cleans up state i.e., running tasks and contexts
+   * pertaining to the evaluator, and adds the evaluator to {@link this#getFailedEvaluators()}, which
+   * can be passed to the {@link org.apache.reef.driver.parameters.DriverRestartFailedEvaluatorHandlers}.
+   * @param evaluator to fail
+   */
+  public void failEvaluator(final MockAllocatedEvaluator evaluator) {
+    if (this.allocatedEvaluators.remove(evaluator)) {
+      this.failedEvaluators.add(new MockFailedEvaluator(evaluator.getId()));
+      // cleanup
+      final Iterator<MockRunningTask> taskIter = this.runningTasks.iterator();
+      while (taskIter.hasNext()) {
+        final MockRunningTask task = taskIter.next();
+        if (task.evaluatorID().equals(evaluator.getId())) {
+          taskIter.remove();
+        }
+      }
+      final Iterator<MockActiveContext> contextIter = this.activeContexts.iterator();
+      while (contextIter.hasNext()) {
+        final MockActiveContext context = contextIter.next();
+        if (context.getEvaluatorId().equals(evaluator.getId())) {
+          contextIter.remove();
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockFailure.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockFailure.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockFailure.java
new file mode 100644
index 0000000..09ad186
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockFailure.java
@@ -0,0 +1,74 @@
+/*
+ * 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.reef.mock.driver;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.evaluator.AllocatedEvaluator;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.wake.time.event.StartTime;
+
+import java.util.Collection;
+
+/**
+ * Used to fail running REEF entities i.e., Evaluators, Contexts, Tasks.
+ */
+@Unstable
+public interface MockFailure {
+
+  /**
+   * @return current Collection of allocated evaluators.
+   */
+  Collection<AllocatedEvaluator> getCurrentAllocatedEvaluators();
+
+  /**
+   * Fail an allocated evaluator.
+   * @param evaluator to be failed
+   */
+  void fail(final AllocatedEvaluator evaluator);
+
+  /**
+   * @return current Collection of active contexts
+   */
+  Collection<ActiveContext> getCurrentActiveContexts();
+
+  /**
+   * Fail an ActiveContext.
+   * @param context to be failed
+   */
+  void fail(final ActiveContext context);
+
+  /**
+   * @return current Collection of running tasks
+   */
+  Collection<RunningTask> getCurrentRunningTasks();
+
+  /**
+   * Fail a running task.
+   * @param task to be failed
+   */
+  void fail(final RunningTask task);
+
+  /**
+   * Fail the driver.
+   * @return the state of the driver at the time of the failure
+   */
+  MockDriverRestartContext failDriver(final int attempt, final StartTime startTime);
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockRuntime.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockRuntime.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockRuntime.java
new file mode 100644
index 0000000..a110ef7
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockRuntime.java
@@ -0,0 +1,92 @@
+/*
+ * 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.reef.mock.driver;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.driver.context.ContextMessage;
+import org.apache.reef.driver.restart.DriverRestartCompleted;
+
+/**
+ * Mock API used to drive the evaluation of ProcessRequest
+ * events, which are triggered by the Application driver.
+ * Clients used this to determine whether a particular ProcessRequest
+ * event should succeed or fail.
+ */
+@Unstable
+public interface MockRuntime extends MockFailure {
+
+  /**
+   * Initiate the start time event to the application driver.
+   */
+  void start();
+
+  /**
+   * Initiate the stop time event to the application driver.
+   */
+  void stop();
+
+  /**
+   * Initiate a driver restart.
+   * @param restartContext contains the state of the driver at the time of failure
+   * @param isTimeout used to fill in {@link DriverRestartCompleted#isTimedOut()}
+   * @param duration recover time duration (added to start time)
+   */
+  void restart(final MockDriverRestartContext restartContext, final boolean isTimeout, final long duration);
+
+  /**
+   * @return true if there is an outstanding ProcessRequest
+   */
+  boolean hasProcessRequest();
+
+  /**
+   * The client (caller) is responsible for determining what
+   * to do with a ProcessRequest event. There are three options:
+   * 1. Pass to the succeed method, which signals success to the driver.
+   * 2. Pass to the fail method, signaling failure to the driver.
+   * 3. Drop it on the floor (e.g., network failure).
+   *
+   * @return the next ProcessRequest object to be processed.
+   */
+  ProcessRequest getNextProcessRequest();
+
+  /**
+   * The driver will be informed that the operation corresponding
+   * to the ProcessRequest succeeded, and will be given any relevant
+   * data structures e.g., AllocatedEvaluator, RunningTask, etc.
+   *
+   * @param request to be processed successfully
+   */
+  void succeed(final ProcessRequest request);
+
+  /**
+   * The driver will be informed that the operation corresponding
+   * to the PRocessRequest failed, and will be given any relevant
+   * data structures e.g., FailedEvaluator, FailedTask, etc.
+   *
+   * @param request to be failed.
+   */
+  void fail(final ProcessRequest request);
+
+  /**
+   * Publish a context message to the application event handlers.
+   * @param contextMessage to be published
+   */
+  void publish(final ContextMessage contextMessage);
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockTaskReturnValueProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockTaskReturnValueProvider.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockTaskReturnValueProvider.java
new file mode 100644
index 0000000..6972a18
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/MockTaskReturnValueProvider.java
@@ -0,0 +1,44 @@
+/*
+ * 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.reef.mock.driver;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.driver.task.CompletedTask;
+import org.apache.reef.mock.driver.runtime.MockRunningTask;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+
+/**
+ * Clients bind an implementation of this interface, which
+ * will be used to create a mock return value for a mock
+ * task execution. This return value will be returned by
+ * the {@link CompletedTask#get()}} method.
+ */
+@Unstable
+@DefaultImplementation(DefaultTaskReturnValueProvider.class)
+public interface MockTaskReturnValueProvider {
+
+  /**
+   * Provide a valid return value for the {@link CompletedTask#get()} method.
+   * @param task that is to be provided with a return value
+   * @return {@link org.apache.reef.task.Task#call(byte[])} return value
+   */
+  byte[] getReturnValue(final MockRunningTask task);
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/ProcessRequest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/ProcessRequest.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/ProcessRequest.java
new file mode 100644
index 0000000..cf4429e
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/ProcessRequest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.reef.mock.driver;
+
+import org.apache.reef.annotations.Unstable;
+
+/**
+ * A ProcessRequest refers to an outstanding event that is
+ * waiting to be processed by the REEF mock runtime. Clients
+ * are responsible for deciding how a ProcessRequest should be
+ * handled, by either:
+ * 1. successfully processing the request
+ * 2. unsucessfully processing the request
+ * 3. dropping the processing request (i.e., loosing it)
+ * These decisions are conveyed through the {MockRuntime} API.
+ */
+@Unstable
+public interface ProcessRequest extends AutoCompletable {
+  /**
+   * process request type.
+   */
+  enum Type {
+    ALLOCATE_EVALUATOR,
+    CLOSE_EVALUATOR,
+    CREATE_CONTEXT,
+    CLOSE_CONTEXT,
+    CREATE_TASK,
+    SUSPEND_TASK,
+    CLOSE_TASK,
+    COMPLETE_TASK,
+    CREATE_CONTEXT_AND_TASK,
+    SEND_MESSAGE_DRIVER_TO_TASK,
+    SEND_MESSAGE_DRIVER_TO_CONTEXT
+  }
+
+  Type getType();
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/package-info.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/package-info.java
new file mode 100644
index 0000000..f9d73f5
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/package-info.java
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Mock runtime API.
+ *
+ * Mock runtime is meant to mimic the semantics of the REEF runtime and
+ * allow:
+ *  1. Applications to driver the forward progress of processing REEF events.
+ *     See {@link org.apache.reef.mock.driver.MockRuntime} API
+ *  2. Control the advancement of the Clock and Alarm callbacks.
+ *     See {@link org.apache.reef.mock.driver.runtime.MockClock}
+ *  3. Inject failures into the REEF applications.
+ *     See {@link org.apache.reef.mock.driver.MockFailure}
+ *
+ * Use {@link org.apache.reef.mock.driver.MockConfiguration} to bind your REEF
+ * driver application event handlers.
+ *
+ * Use {@link org.apache.reef.mock.driver.MockRuntime#start()} to trigger the
+ * driver start event and {@link org.apache.reef.mock.driver.MockRuntime#stop()}}
+ * or {@link org.apache.reef.mock.driver.runtime.MockClock#close()} to trigger the driver
+ * stop event.
+ */
+package org.apache.reef.mock.driver;

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/AllocateEvaluator.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/AllocateEvaluator.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/AllocateEvaluator.java
new file mode 100644
index 0000000..d717fc6
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/AllocateEvaluator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockAllocatedEvaluator;
+import org.apache.reef.mock.driver.runtime.MockFailedEvaluator;
+
+/**
+ * Allocate Evaluator process request.
+ */
+@Unstable
+@Private
+public final class AllocateEvaluator implements
+    ProcessRequestInternal<MockAllocatedEvaluator, FailedEvaluator> {
+
+  private final MockAllocatedEvaluator evaluator;
+
+  public AllocateEvaluator(final MockAllocatedEvaluator evaluator) {
+    this.evaluator = evaluator;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.ALLOCATE_EVALUATOR;
+  }
+
+  @Override
+  public MockAllocatedEvaluator getSuccessEvent() {
+    return this.evaluator;
+  }
+
+  @Override
+  public FailedEvaluator getFailureEvent() {
+    return new MockFailedEvaluator(evaluator.getId());
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return false;
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseContext.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseContext.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseContext.java
new file mode 100644
index 0000000..d6408d7
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseContext.java
@@ -0,0 +1,76 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.ClosedContext;
+import org.apache.reef.driver.context.FailedContext;
+import org.apache.reef.mock.driver.AutoCompletable;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockActiveContext;
+import org.apache.reef.mock.driver.runtime.MockClosedContext;
+import org.apache.reef.mock.driver.runtime.MockFailedContext;
+
+/**
+ * close context process request.
+ */
+@Unstable
+@Private
+public final class CloseContext implements
+    ProcessRequestInternal<ClosedContext, FailedContext>,
+    AutoCompletable {
+
+  private final MockActiveContext context;
+
+  public CloseContext(final MockActiveContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.CLOSE_CONTEXT;
+  }
+
+  @Override
+  public MockClosedContext getSuccessEvent() {
+    return new MockClosedContext(this.context);
+  }
+
+  @Override
+  public FailedContext getFailureEvent() {
+    return new MockFailedContext(this.context);
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return !this.context.getParentContext().isPresent();
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    return new CloseEvaluator(this.context.getEvaluator());
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseEvaluator.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseEvaluator.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseEvaluator.java
new file mode 100644
index 0000000..c329873
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseEvaluator.java
@@ -0,0 +1,78 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.CompletedEvaluator;
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockAllocatedEvaluator;
+import org.apache.reef.mock.driver.runtime.MockFailedEvaluator;
+
+/**
+ * close evaluator request.
+ */
+@Unstable
+@Private
+public final class CloseEvaluator implements ProcessRequestInternal<CompletedEvaluator, FailedEvaluator> {
+
+  private final MockAllocatedEvaluator evaluator;
+
+  public CloseEvaluator(final MockAllocatedEvaluator evaluator) {
+    this.evaluator = evaluator;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.CLOSE_EVALUATOR;
+  }
+
+  @Override
+  public CompletedEvaluator getSuccessEvent() {
+    return new CompletedEvaluator() {
+      @Override
+      public String getId() {
+        return evaluator.getId();
+      }
+    };
+  }
+
+  @Override
+  public FailedEvaluator getFailureEvent() {
+    // TODO[initialize remaining failed contstructer fields]
+    return new MockFailedEvaluator(evaluator.getId());
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return false;
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseTask.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseTask.java
new file mode 100644
index 0000000..c726d94
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CloseTask.java
@@ -0,0 +1,89 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.task.CompletedTask;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.mock.driver.MockTaskReturnValueProvider;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockCompletedTask;
+import org.apache.reef.mock.driver.runtime.MockRunningTask;
+import org.apache.reef.util.Optional;
+
+/**
+ * close task process request.
+ */
+@Unstable
+@Private
+public final class CloseTask implements ProcessRequestInternal<CompletedTask, FailedTask> {
+
+  private final MockRunningTask task;
+
+  private final MockTaskReturnValueProvider taskReturnValueProvider;
+
+  public CloseTask(
+      final MockRunningTask task,
+      final MockTaskReturnValueProvider taskReturnValueProvider) {
+    this.task = task;
+    this.taskReturnValueProvider = taskReturnValueProvider;
+  }
+
+  public MockRunningTask getTask() {
+    return task;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.CLOSE_TASK;
+  }
+
+  @Override
+  public MockCompletedTask getSuccessEvent() {
+    return new MockCompletedTask(this.task, this.taskReturnValueProvider.getReturnValue(task));
+  }
+
+  @Override
+  public FailedTask getFailureEvent() {
+    return new FailedTask(
+      task.getId(),
+      "mock",
+      Optional.<String>empty(),
+      Optional.<Throwable>empty(),
+      Optional.<byte[]>empty(),
+      Optional.of(this.task.getActiveContext()));
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return false;
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CompleteTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CompleteTask.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CompleteTask.java
new file mode 100644
index 0000000..c38c10b
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CompleteTask.java
@@ -0,0 +1,82 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.task.CompletedTask;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.mock.driver.MockTaskReturnValueProvider;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockCompletedTask;
+import org.apache.reef.mock.driver.runtime.MockRunningTask;
+
+/**
+ * close task process request.
+ */
+@Unstable
+@Private
+public final class CompleteTask implements ProcessRequestInternal<CompletedTask, FailedTask> {
+
+  private final MockRunningTask task;
+
+  private final MockTaskReturnValueProvider returnValueProvider;
+
+  public CompleteTask(
+      final MockRunningTask task,
+      final MockTaskReturnValueProvider returnValueProvider) {
+    this.task = task;
+    this.returnValueProvider = returnValueProvider;
+  }
+
+  public MockRunningTask getTask() {
+    return task;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.COMPLETE_TASK;
+  }
+
+  @Override
+  public CompletedTask getSuccessEvent() {
+    return new MockCompletedTask(this.task, this.returnValueProvider.getReturnValue(task));
+  }
+
+  @Override
+  public FailedTask getFailureEvent() {
+    return null;
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return false;
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContext.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContext.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContext.java
new file mode 100644
index 0000000..138d070
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContext.java
@@ -0,0 +1,76 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.FailedContext;
+import org.apache.reef.mock.driver.AutoCompletable;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockActiveContext;
+import org.apache.reef.mock.driver.runtime.MockFailedContext;
+
+/**
+ * create context process request.
+ */
+@Unstable
+@Private
+public final class CreateContext implements
+    ProcessRequestInternal<MockActiveContext, FailedContext>,
+    AutoCompletable {
+
+  private final MockActiveContext context;
+
+  private boolean autoComplete = false;
+
+  public CreateContext(final MockActiveContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.CREATE_CONTEXT;
+  }
+
+  @Override
+  public MockActiveContext getSuccessEvent() {
+    return this.context;
+  }
+
+  @Override
+  public FailedContext getFailureEvent() {
+    return new MockFailedContext(this.context);
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return this.autoComplete;
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    this.autoComplete = value;
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    return new CloseContext(this.context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContextAndTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContextAndTask.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContextAndTask.java
new file mode 100644
index 0000000..7d724a0
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateContextAndTask.java
@@ -0,0 +1,98 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.io.Tuple;
+import org.apache.reef.mock.driver.AutoCompletable;
+import org.apache.reef.mock.driver.MockTaskReturnValueProvider;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockActiveContext;
+import org.apache.reef.mock.driver.runtime.MockFailedContext;
+import org.apache.reef.mock.driver.runtime.MockRunningTask;
+import org.apache.reef.util.Optional;
+
+/**
+ * create context and task process request.
+ */
+@Unstable
+@Private
+public final class CreateContextAndTask implements
+    ProcessRequestInternal<Tuple<MockActiveContext, MockRunningTask>, Tuple<MockFailedContext, FailedTask>>,
+    AutoCompletable {
+
+  private final MockActiveContext context;
+
+  private final MockRunningTask task;
+
+  private final MockTaskReturnValueProvider taskReturnValueProvider;
+
+  private boolean autoComplete = true;
+
+  public CreateContextAndTask(
+      final MockActiveContext context,
+      final MockRunningTask task,
+      final MockTaskReturnValueProvider taskReturnValueProvider) {
+    this.context = context;
+    this.task = task;
+    this.taskReturnValueProvider = taskReturnValueProvider;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.CREATE_CONTEXT_AND_TASK;
+  }
+
+  @Override
+  public Tuple<MockActiveContext, MockRunningTask> getSuccessEvent() {
+    return new Tuple<>(this.context, this.task);
+  }
+
+  @Override
+  public Tuple<MockFailedContext, FailedTask> getFailureEvent() {
+    return new Tuple<>(
+        new MockFailedContext(this.context),
+        new FailedTask(
+            this.task.getId(),
+            "mock",
+            Optional.<String>empty(),
+            Optional.<Throwable>empty(),
+            Optional.<byte[]>empty(),
+            Optional.of((ActiveContext)this.context)));
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return this.autoComplete;
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    this.autoComplete = value;
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    return new CompleteTask(this.task, this.taskReturnValueProvider);
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateTask.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateTask.java
new file mode 100644
index 0000000..699b0ae
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/CreateTask.java
@@ -0,0 +1,89 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.mock.driver.AutoCompletable;
+import org.apache.reef.mock.driver.MockTaskReturnValueProvider;
+import org.apache.reef.mock.driver.ProcessRequest;
+import org.apache.reef.mock.driver.runtime.MockRunningTask;
+import org.apache.reef.util.Optional;
+
+/**
+ * create task process request.
+ */
+@Unstable
+@Private
+public final class CreateTask implements
+    ProcessRequestInternal<RunningTask, FailedTask>,
+    AutoCompletable {
+
+  private final MockRunningTask task;
+
+  private final MockTaskReturnValueProvider returnValueProvider;
+
+  private boolean autoComplete = true;
+
+  public CreateTask(
+      final MockRunningTask task,
+      final MockTaskReturnValueProvider returnValueProvider) {
+    this.task = task;
+    this.returnValueProvider = returnValueProvider;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.CREATE_TASK;
+  }
+
+  @Override
+  public boolean doAutoComplete() {
+    return this.autoComplete;
+  }
+
+  @Override
+  public ProcessRequest getCompletionProcessRequest() {
+    return new CompleteTask(this.task, this.returnValueProvider);
+  }
+
+  @Override
+  public void setAutoComplete(final boolean value) {
+    this.autoComplete = value;
+  }
+
+  @Override
+  public MockRunningTask getSuccessEvent() {
+    return this.task;
+  }
+
+  @Override
+  public FailedTask getFailureEvent() {
+    return new FailedTask(
+        this.task.getId(),
+        "mock",
+        Optional.<String>empty(),
+        Optional.<Throwable>empty(),
+        Optional.<byte[]>empty(),
+        Optional.of(this.task.getActiveContext()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/5ed56eba/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/ProcessRequestInternal.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/ProcessRequestInternal.java b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/ProcessRequestInternal.java
new file mode 100644
index 0000000..d8509d6
--- /dev/null
+++ b/lang/java/reef-runtime-mock/src/main/java/org/apache/reef/mock/driver/request/ProcessRequestInternal.java
@@ -0,0 +1,44 @@
+/*
+ * 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.reef.mock.driver.request;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.mock.driver.ProcessRequest;
+
+/**
+ * internal process request API.
+ * @param <S> successful event
+ * @param <F> failure event
+ */
+@Unstable
+@Private
+public interface ProcessRequestInternal<S, F> extends ProcessRequest {
+
+  /**
+   * @return the outcome of a successful processing of this request
+   */
+  S getSuccessEvent();
+
+  /**
+   * @return the outcome of an unsuccessful processing of this request
+   */
+  F getFailureEvent();
+}