You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by mo...@apache.org on 2018/07/06 23:59:39 UTC

[reef] branch master updated: [REEF-2025] A new module containing the new Java bridge (#1466)

This is an automated email from the ASF dual-hosted git repository.

motus pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/reef.git


The following commit(s) were added to refs/heads/master by this push:
     new c4f934f  [REEF-2025] A new module containing the new Java bridge (#1466)
c4f934f is described below

commit c4f934f044fdacb3845ce3a083e940249d6de693
Author: Tyson Condie <tc...@users.noreply.github.com>
AuthorDate: Fri Jul 6 16:59:37 2018 -0700

    [REEF-2025] A new module containing the new Java bridge (#1466)
    
    This update introduces a new Java bridge for Drivers implemented in alternative languages. It provides the following artifacts (note: client driver refers to the application driver implemented in an alternative programming language):
    
    	1. A generic framework for passing information between the Java driver and the client driver.
    	2. A gRPC based implementation of the bridge that passes information via protocol buffers over gRPC.
    	3. Protocol buffer definitions for all information that flows between the Java driver and the client driver.
    	4. A Java implementation of the driver client that can be used for developing unit tests and serve as a template for implementing a driver client (say in C#).
    	5. Test cases to fail based unit tests that cover the Java bridge and client.
    
    JIRA: [REEF-2025](https://issues.apache.org/jira/browse/REEF-2025)
    
    Pull Request:
        Closes #1466
---
 .gitignore                                         |   1 +
 lang/common/proto/bridge/ClientProtocol.proto      | 127 +++
 .../common/proto/bridge/DriverClientProtocol.proto | 224 +++++
 .../proto/bridge/DriverCommonProtocol.proto}       |  41 +-
 .../proto/bridge/DriverServiceProtocol.proto       | 166 ++++
 lang/java/reef-bridge-proto-java/pom.xml           | 297 +++++++
 .../reef/bridge/client/DriverServiceLauncher.java  | 197 +++++
 .../apache/reef/bridge/client/package-info.java}   |  17 +-
 .../driver/client/AlarmDispatchHandler.java}       |  21 +-
 .../client/DefaultDriverClientStopHandler.java}    |  34 +-
 .../bridge/driver/client/DriverClientClock.java    | 132 +++
 .../driver/client/DriverClientConfiguration.java   | 204 +++++
 .../driver/client/DriverClientDispatcher.java      | 364 ++++++++
 .../client/DriverClientEvaluatorRequestor.java     |  67 ++
 .../client/DriverClientExceptionHandler.java       |  50 ++
 .../bridge/driver/client/DriverClientService.java} |  38 +-
 .../bridge/driver/client/DriverServiceClient.java  | 140 +++
 .../bridge/driver/client/JVMClientProcess.java     | 120 +++
 .../driver/client/JavaDriverClientLauncher.java    | 227 +++++
 .../driver/client/events/ActiveContextBridge.java  | 113 +++
 .../client/events/AllocatedEvaluatorBridge.java    | 146 +++
 .../driver/client/events/ClosedContextBridge.java  |  77 ++
 .../client/events/CompletedEvaluatorBridge.java}   |  30 +-
 .../driver/client/events/CompletedTaskBridge.java  |  61 ++
 .../driver/client/events/ContextMessageBridge.java |  69 ++
 .../driver/client/events/FailedContextBridge.java  | 112 +++
 .../client/events/FailedEvaluatorBridge.java       |  75 ++
 .../driver/client/events/RunningTaskBridge.java    |  89 ++
 .../driver/client/events/SuspendedTaskBridge.java} |  47 +-
 .../driver/client/events/TaskMessageBridge.java    |  78 ++
 .../bridge/driver/client/events/package-info.java} |  17 +-
 .../client/grpc/DriverClientGrpcConfiguration.java |  48 +
 .../client/grpc/GRPCDriverClientService.java       | 629 +++++++++++++
 .../client/grpc/GRPCDriverServiceClient.java       | 258 ++++++
 .../bridge/driver/client/grpc/package-info.java}   |  17 +-
 .../parameters/DriverRegistrationTimeout.java}     |  19 +-
 .../client/grpc/parameters/DriverServicePort.java} |  18 +-
 .../client/grpc/parameters/package-info.java}      |  17 +-
 .../reef/bridge/driver/client/package-info.java}   |  17 +-
 .../parameters/ClientDriverStopHandler.java}       |  25 +-
 .../DriverClientDispatchThreadCount.java}          |  19 +-
 .../driver/client/parameters/package-info.java}    |  17 +-
 .../reef/bridge/driver/common/grpc/GRPCUtils.java  | 127 +++
 .../bridge/driver/common/grpc/ObserverCleanup.java |  55 ++
 .../bridge/driver/common/grpc/package-info.java}   |  17 +-
 .../reef/bridge/driver/common/package-info.java}   |  17 +-
 .../driver/launch/BridgeDriverLauncher.java}       |  27 +-
 .../driver/launch/azbatch/AzureBatchLauncher.java  |  76 ++
 .../driver/launch/azbatch/package-info.java}       |  17 +-
 .../bridge/driver/launch/local/LocalLauncher.java  |  70 ++
 .../bridge/driver/launch/local/package-info.java}  |  17 +-
 .../reef/bridge/driver/launch/package-info.java}   |  17 +-
 .../bridge/driver/launch/yarn/YarnLauncher.java    |  82 ++
 .../bridge/driver/launch/yarn/package-info.java}   |  17 +-
 .../driver/service/DriverClientException.java}     |  22 +-
 .../reef/bridge/driver/service/DriverService.java  | 172 ++++
 .../driver/service/DriverServiceConfiguration.java | 124 +++
 .../DriverServiceConfigurationProvider.java}       |  25 +-
 .../driver/service/DriverServiceHandlers.java      | 304 +++++++
 .../driver/service/DriverStatusHTTPHandler.java    | 200 +++++
 .../driver/service/grpc/GRPCDriverService.java     | 976 +++++++++++++++++++++
 .../GRPCDriverServiceConfigurationProvider.java    | 156 ++++
 .../bridge/driver/service/grpc/package-info.java}  |  17 +-
 .../reef/bridge/driver/service/package-info.java}  |  17 +-
 .../service/parameters/DriverClientCommand.java}   |  20 +-
 .../parameters/HTTPStatusAlarmInterval.java}       |  22 +-
 .../parameters/HTTPStatusNumberOfRetries.java}     |  22 +-
 .../driver/service/parameters/package-info.java}   |  17 +-
 .../reef/bridge/examples/hello/HelloREEF.java      |  71 ++
 .../reef/bridge/examples/hello/package-info.java}  |  17 +-
 .../apache/reef/bridge/examples/package-info.java} |  17 +-
 .../driver/evaluator/EvaluatorDescriptorImpl.java  |   2 +-
 .../reef/runtime/common/files/REEFFileNames.java   |  16 +
 .../common/utils/DefaultExceptionCodec.java        |  10 +-
 lang/java/reef-tests/pom.xml                       |  11 +
 .../reef/tests/fail/driver/FailBridgeClient.java   | 101 +++
 .../apache/reef/tests/fail/task/BridgeClient.java  |  92 ++
 .../tests/fail/util/FailBridgeClientUtils.java     | 102 +++
 .../apache/reef/tests/fail/util/package-info.java} |  17 +-
 .../reef/tests/fail/FailBridgeDriverTest.java      | 144 +++
 .../apache/reef/tests/fail/FailBridgeTaskTest.java |  98 +++
 .../org/apache/reef/tests/fail/FailTestSuite.java  |   4 +-
 pom.xml                                            |  34 +-
 83 files changed, 7363 insertions(+), 471 deletions(-)

diff --git a/.gitignore b/.gitignore
index b4032c4..1c6af07 100644
--- a/.gitignore
+++ b/.gitignore
@@ -11,6 +11,7 @@ target
 generated
 build
 StyleCop.Cache
+dependency-reduced-pom.xml
 #
 # ----------------------------------------------------------------------
 # IDE settings
diff --git a/lang/common/proto/bridge/ClientProtocol.proto b/lang/common/proto/bridge/ClientProtocol.proto
new file mode 100644
index 0000000..3013b2f
--- /dev/null
+++ b/lang/common/proto/bridge/ClientProtocol.proto
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+syntax = "proto3";
+
+option java_package = "org.apache.reef.bridge.proto";
+option java_outer_classname = "ClientProtocol";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Proto";
+
+package driverbridge;
+
+/*
+ * Local runtime parameters.
+ */
+message LocalRuntimeParameters {
+  uint32 max_number_of_evaluators = 1;
+  string runtime_root_folder = 2;
+  float jvm_heap_slack = 3;
+  repeated string rack_names = 4;
+}
+
+/*
+ * Yarn runtime parameters.
+ */
+message YarnRuntimeParameters {
+  string queue = 1;
+  uint32 priority = 2;
+  bool unmanged_driver = 3;
+
+  // providers
+  string filesystem_url = 5;
+  string job_submission_directory_prefix = 6;
+}
+
+/*
+ * Azure batch runtime parameters.
+ */
+message AzureBatchRuntimeParameters {
+  // Azure Batch Account Information
+  string azure_batch_account_name = 1;
+
+  string azure_batch_account_key = 2;
+
+  string azure_batch_account_uri = 3;
+
+  string azure_batch_pool_id = 4;
+
+  // Azure Storage Account Information
+  string azure_storage_account_name = 10;
+
+  string azure_storage_account_key = 11;
+
+  string azure_storage_container_name = 12;
+}
+
+/*
+ * Mesos runtime parameters.
+ */
+message MesosRuntimeParameters {
+}
+
+/*
+ * Parameters related to the creation of a driver runtime.
+ */
+message DriverClientConfiguration {
+  string jobid = 1;
+
+  // driver machine resources
+  uint32 cpu_cores = 2;
+  uint32 memory_mb = 3;
+
+  string driver_job_submission_directory = 4;
+
+  // the runtime on which to launch
+  oneof runtime {
+    LocalRuntimeParameters local_runtime = 5;
+    YarnRuntimeParameters yarn_runtime = 6;
+    AzureBatchRuntimeParameters azbatch_runtime = 7;
+    MesosRuntimeParameters mesos_runtime = 8;
+  }
+
+  // The command to launch the driver client
+  string driver_client_launch_command = 10;
+
+  // Enable driver restart?
+  bool driver_restart_enable = 11;
+
+  // Driver restart evaluator recovery seconds (optional)
+  uint32 driver_restart_evaluator_recovery_seconds = 12;
+
+  // TCP port range
+  uint32 tcp_port_range_begin = 15;
+  uint32 tcp_port_range_count = 16;
+  uint32 tcp_port_range_try_count = 17;
+
+  // file dependencies
+  repeated string global_files = 20;
+  repeated string local_files = 21;
+  repeated string global_libraries = 22;
+  repeated string local_libraries = 23;
+
+  // enable http driver
+  bool enable_http_driver = 25;
+
+  // General information
+  enum OS {
+    WINDOWS = 0;
+    LINUX = 1;
+  }
+  OS operating_system = 30;
+}
\ No newline at end of file
diff --git a/lang/common/proto/bridge/DriverClientProtocol.proto b/lang/common/proto/bridge/DriverClientProtocol.proto
new file mode 100644
index 0000000..7411e8d
--- /dev/null
+++ b/lang/common/proto/bridge/DriverClientProtocol.proto
@@ -0,0 +1,224 @@
+/*
+ * 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.
+ */
+
+syntax = "proto3";
+
+// option java_generic_services = true;
+option java_multiple_files = true;
+option java_package = "org.apache.reef.bridge.proto";
+option java_outer_classname = "DriverClientProtocol";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Proto";
+
+package driverbridge;
+
+import "DriverCommonProtocol.proto";
+
+// The java driver service definition.
+service DriverClient {
+  // Inquire if idle
+  rpc IdlenessCheckHandler (Void) returns (IdleStatus) {}
+
+  // Request for resources
+  rpc StartHandler (StartTimeInfo) returns (Void) {}
+
+  rpc StopHandler (StopTimeInfo) returns (ExceptionInfo) {}
+
+  rpc AlarmTrigger (AlarmTriggerInfo) returns (Void) {}
+
+  // Evaluator handlers
+  rpc AllocatedEvaluatorHandler (EvaluatorInfo) returns (Void) {}
+
+  rpc CompletedEvaluatorHandler (EvaluatorInfo) returns (Void) {}
+
+  rpc FailedEvaluatorHandler (EvaluatorInfo) returns (Void) {}
+
+  // Context handlers
+  rpc ActiveContextHandler (ContextInfo) returns (Void) {}
+
+  rpc ClosedContextHandler (ContextInfo) returns (Void) {}
+
+  rpc FailedContextHandler (ContextInfo) returns (Void) {}
+
+  rpc ContextMessageHandler (ContextMessageInfo) returns (Void) {}
+
+  // Task handlers
+  rpc RunningTaskHandler (TaskInfo) returns (Void) {}
+
+  rpc FailedTaskHandler (TaskInfo) returns (Void) {}
+
+  rpc CompletedTaskHandler (TaskInfo) returns (Void) {}
+
+  rpc SuspendedTaskHandler (TaskInfo) returns (Void) {}
+
+  rpc TaskMessageHandler (TaskMessageInfo) returns (Void) {}
+
+  // Client Handlers
+  rpc ClientMessageHandler (ClientMessageInfo) returns (Void) {}
+
+  rpc ClientCloseHandler (Void) returns (Void) {}
+
+  rpc ClientCloseWithMessageHandler (ClientMessageInfo) returns (Void) {}
+
+  // Driver Restart Handlers
+  rpc DriverRestartHandler (DriverRestartInfo) returns (Void) {}
+
+  rpc DriverRestartActiveContextHandler (ContextInfo) returns (Void) {}
+
+  rpc DriverRestartRunningTaskHandler (TaskInfo) returns (Void) {}
+
+  rpc DriverRestartCompletedHandler (DriverRestartCompletedInfo) returns (Void) {}
+
+  rpc DriverRestartFailedEvaluatorHandler (EvaluatorInfo) returns (Void) {}
+}
+
+// Driver restart information
+message DriverRestartInfo {
+  uint32 resubmission_attempts = 1;
+
+  StartTimeInfo start_time = 2;
+
+  repeated string expected_evaluator_ids = 3;
+}
+
+// Driver restart completed information
+message DriverRestartCompletedInfo {
+  StopTimeInfo completion_time = 1;
+
+  bool is_timed_out = 2;
+}
+
+// IdleStatus response to idleness inquiry
+message IdleStatus {
+  bool is_idle = 1;
+  string reason = 2;
+}
+
+// The request message containing resource request.
+message StartTimeInfo {
+  int64 start_time = 1;
+}
+
+message StopTimeInfo {
+  int64 stop_time = 1;
+}
+
+// Information associated with an alarm that was set.
+message AlarmTriggerInfo {
+  string alarm_id = 1;
+}
+
+// Evaluator descriptor information.
+message EvaluatorDescriptorInfo {
+  // the amount of memory allocated
+  int32 memory = 1;
+
+  // the number of virtual cores allocated
+  int32 cores = 2;
+
+  // name of the runtime
+  string runtime_name = 3;
+
+  // node descriptor info
+  message NodeDescriptorInfo {
+      string id = 1;
+
+      string ip_address = 5;
+      string host_name = 6;
+      int32 port = 7;
+
+      // rack info
+      string rack_name = 10;
+  }
+  NodeDescriptorInfo node_descriptor_info = 5;
+}
+
+// Information related to an evaluator.
+message EvaluatorInfo {
+  string evaluator_id = 1;
+
+  message FailureInfo {
+     string message = 1;
+     repeated string failed_contexts = 2;
+     string failed_task_id = 3;
+  }
+  FailureInfo failure = 2;
+
+  EvaluatorDescriptorInfo descriptor_info = 3;
+}
+
+// Information associated with a context.
+message ContextInfo {
+  string context_id = 1;
+
+  string evaluator_id = 2;
+
+  string parent_id = 3;
+
+  // Carry this with us for driver restart
+  EvaluatorDescriptorInfo evaluator_descriptor_info = 4;
+
+  // Optional exception information
+  ExceptionInfo exception = 5;
+}
+
+// Context message information.
+message ContextMessageInfo {
+  string context_id = 1;
+
+  bytes payload = 2;
+
+  int64 sequence_number = 3;
+
+  string message_source_id = 4;
+}
+
+// Task information
+message TaskInfo {
+  // Task identifier.
+  string task_id = 1;
+
+  // Task result.
+  bytes result = 2;
+
+  /* Carry entire context info since client may not have received it
+   * when submitting task against allocated evalautor.
+   */
+  ContextInfo context = 5;
+
+  // Possible exception encountered in task execution.
+  ExceptionInfo exception = 10;
+}
+
+// Task message information.
+message TaskMessageInfo {
+  string task_id = 1;
+
+  bytes payload = 2;
+
+  int64 sequence_number = 3;
+
+  string context_id = 4;
+
+  string message_source_id = 5;
+}
+
+// Client message information.
+message ClientMessageInfo {
+  bytes payload = 1;
+}
\ No newline at end of file
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/common/proto/bridge/DriverCommonProtocol.proto
similarity index 57%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/common/proto/bridge/DriverCommonProtocol.proto
index 9967d9b..439b203 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/common/proto/bridge/DriverCommonProtocol.proto
@@ -16,20 +16,33 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+syntax = "proto3";
 
-/**
- * Test suite of tests covering failure scenarios.
- */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+// option java_generic_services = true;
+option java_multiple_files = true;
+option java_package = "org.apache.reef.bridge.proto";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Proto";
+
+package driverbridge;
+
+// Void message type.
+message Void {}
+
+// Exception information.
+message ExceptionInfo {
+    // no error present if true
+    bool no_error = 1;
+
+    // Exception name/type
+    string name = 2;
+
+    // Exception message
+    string message = 3;
+
+    // Stack trace
+    repeated string stack_trace = 4;
+
+    // Data associated with exception
+    bytes data = 5;
 }
diff --git a/lang/common/proto/bridge/DriverServiceProtocol.proto b/lang/common/proto/bridge/DriverServiceProtocol.proto
new file mode 100644
index 0000000..d9f8ffa
--- /dev/null
+++ b/lang/common/proto/bridge/DriverServiceProtocol.proto
@@ -0,0 +1,166 @@
+/*
+ * 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.
+ */
+
+syntax = "proto3";
+
+// option java_generic_services = true;
+option java_multiple_files = true;
+option java_package = "org.apache.reef.bridge.proto";
+option java_outer_classname = "DriverBridgeProtocol";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Proto";
+
+package driverbridge;
+
+import "DriverCommonProtocol.proto";
+
+// The java driver service definition.
+service DriverService {
+  // Driver client registration
+  rpc RegisterDriverClient (DriverClientRegistration) returns (Void) {}
+
+  // Request for resources
+  rpc RequestResources (ResourceRequest) returns (Void) {}
+
+  // Request system shutdown
+  rpc Shutdown (ShutdownRequest) returns (Void) {}
+
+  // Request for setting an alarm
+  rpc SetAlarm (AlarmRequest) returns (Void) {}
+
+  // Request operation on an allocated evaluator
+  rpc AllocatedEvaluatorOp (AllocatedEvaluatorRequest) returns (Void) {}
+
+  // Request operation on an active context
+  rpc ActiveContextOp (ActiveContextRequest) returns (Void) {}
+
+  // Request operation on a running task
+  rpc RunningTaskOp (RunningTaskRequest) returns (Void) {}
+}
+
+// Driver client registration information.
+message DriverClientRegistration {
+  // The client's host
+  string host = 1;
+
+  // The client's server port
+  int32 port = 2;
+
+  // Error during initialization
+  ExceptionInfo exception = 5;
+}
+
+// The request message containing resource request.
+message ResourceRequest {
+  repeated string node_name_list = 1;
+
+  repeated string rack_name_list = 2;
+
+  int32 resource_count = 3;
+
+  int32 memory_size = 4;
+
+  int32 priority = 5;
+
+  int32 cores = 6;
+
+  bool relax_locality = 7;
+
+  string runtime_name = 8;
+}
+
+// Request for an alarm to be set
+message AlarmRequest {
+  // used to uniquely identify the alarm
+  string alarm_id = 1;
+
+  // timeout in milliseconds
+  int32 timeout_ms = 2;
+}
+
+// Shutdown request.
+message ShutdownRequest {
+  ExceptionInfo exception = 1;
+}
+
+// Allocated evaluator request.
+message AllocatedEvaluatorRequest {
+  // The evaluator used to submit
+  string evaluator_id = 1;
+
+  bool close_evaluator = 2;
+
+  repeated string add_files = 3;
+
+  repeated string add_libraries = 4;
+
+  // Evaluator configuration
+  string evaluator_configuration = 5;
+
+  // Context configuration
+  string context_configuration = 6;
+
+  // Task configuration
+  string task_configuration = 7;
+
+  message EvaluatorProcessRequest {
+    int32 memory_mb = 1;
+
+    string configuration_file_name = 2;
+
+    string standard_out = 3;
+
+    string standard_err = 4;
+
+    repeated string options = 5;
+  }
+  EvaluatorProcessRequest set_process = 8;
+}
+
+// Active context request.
+message ActiveContextRequest {
+  string context_id = 1;
+
+  oneof operation {
+    // close the context
+    bool close_context = 2;
+
+    // send message to the context
+    bytes message = 3;
+
+    // create a child context
+    string new_context_request = 4;
+
+    // launch a task
+    string new_task_request = 5;
+  }
+}
+
+// Running task request.
+message RunningTaskRequest {
+  string task_id = 1;
+
+  bytes message = 2;
+
+  enum Operation {
+    CLOSE = 0;
+    SUSPEND = 1;
+    SEND_MESSAGE = 2;
+  }
+  Operation operation = 5;
+}
diff --git a/lang/java/reef-bridge-proto-java/pom.xml b/lang/java/reef-bridge-proto-java/pom.xml
new file mode 100644
index 0000000..8cc3a52
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/pom.xml
@@ -0,0 +1,297 @@
+<?xml version="1.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.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>reef-bridge-proto-java</artifactId>
+    <name>REEF Bridge Protobuf Java</name>
+    <description>Protocol Buffer Bridge between JVM and CLR.</description>
+
+    <parent>
+        <groupId>org.apache.reef</groupId>
+        <artifactId>reef-project</artifactId>
+        <version>0.17.0-SNAPSHOT</version>
+        <relativePath>../../..</relativePath>
+    </parent>
+
+    <properties>
+        <rootPath>${basedir}/../../..</rootPath>
+        <protoPath>${rootPath}/lang/common/proto/bridge</protoPath>
+        <!-- protobuf paths -->
+        <protobuf.input.directory>${protoPath}</protobuf.input.directory>
+        <protobuf.output.directory>${project.build.directory}/generated-sources</protobuf.output.directory>
+        <!-- override parent versions -->
+        <protobuf.version>3.5.1</protobuf.version>
+        <netty.version>4.1.25.Final</netty.version>
+    </properties>
+
+    <dependencies>
+        <!-- REEF -->
+        <dependency>
+            <groupId>org.apache.reef</groupId>
+            <artifactId>reef-annotations</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>reef-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>reef-runtime-local</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>reef-runtime-yarn</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>reef-runtime-azbatch</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>reef-webserver</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>reef-io</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.reef</groupId>
+            <artifactId>reef-examples</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <!-- END REEF -->
+        <!-- gRPC -->
+        <dependency>
+            <groupId>com.google.protobuf</groupId>
+            <artifactId>protobuf-java</artifactId>
+            <version>${protobuf.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+            <version>${netty.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        <!-- END gRPC -->
+        <!-- HADOOP -->
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <!-- End of HADOOP -->
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <extensions>
+            <!-- provides os.detected.classifier (i.e. linux-x86_64, osx-x86_64) property -->
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>${os-maven-plugin.version}</version>
+            </extension>
+        </extensions>
+
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+                <configuration>
+                    <configLocation>lang/java/reef-common/src/main/resources/checkstyle-strict.xml</configLocation>
+                </configuration>
+            </plugin>
+            <!-- Generate gRPC definitions -->
+            <plugin>
+                <groupId>org.xolstice.maven.plugins</groupId>
+                <artifactId>protobuf-maven-plugin</artifactId>
+                <version>${xolstice.version}</version>
+                <configuration>
+                    <pluginId>grpc-java</pluginId>
+                    <pluginArtifact>io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+                    <protoSourceRoot>${protobuf.input.directory}</protoSourceRoot>
+                    <outputDirectory>${protobuf.output.directory}</outputDirectory>
+                    <clearOutputDirectory>true</clearOutputDirectory>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>compile-custom</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Generate protocol buffer message definitions -->
+            <plugin>
+                <groupId>org.xolstice.maven.plugins</groupId>
+                <artifactId>protobuf-maven-plugin</artifactId>
+                <version>${xolstice.version}</version>
+                <configuration>
+                    <pluginId>grpc-java</pluginId>
+                    <protocArtifact>com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+                    <protoSourceRoot>${protobuf.input.directory}</protoSourceRoot>
+                    <outputDirectory>${protobuf.output.directory}</outputDirectory>
+                    <clearOutputDirectory>false</clearOutputDirectory>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- add generated grpc classes into the package -->
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>${build-helper-maven-plugin.version}</version>
+                <executions>
+                    <execution>
+                        <id>add-classes</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>${protobuf.output.directory}</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!--  shade protobuf to avoid version conflicts -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>${maven-shade-plugin.version}</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <createDependencyReducedPom>false</createDependencyReducedPom>
+                            <relocations>
+                                <relocation>
+                                    <pattern>com.google.protobuf</pattern>
+                                    <shadedPattern>${project.groupId}.${project.artifactId}.shaded.protobuf</shadedPattern>
+                                </relocation>
+                            </relocations>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>${maven-shade-plugin.version}</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <createDependencyReducedPom>false</createDependencyReducedPom>
+                            <relocations>
+                                <relocation>
+                                    <pattern>io.netty</pattern>
+                                    <shadedPattern>${project.groupId}.${project.artifactId}.shaded.netty</shadedPattern>
+                                </relocation>
+                            </relocations>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>${maven.assembly}</version>
+                <configuration>
+                    <descriptorRefs>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                    </descriptorRefs>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id> <!-- this is used for inheritance merges -->
+                        <phase>package</phase> <!-- bind to the packaging phase -->
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>findbugs-maven-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java
new file mode 100644
index 0000000..a974383
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java
@@ -0,0 +1,197 @@
+/*
+ * 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.bridge.client;
+
+import com.google.protobuf.util.JsonFormat;
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.annotations.audience.ClientSide;
+import org.apache.reef.bridge.driver.launch.BridgeDriverLauncher;
+import org.apache.reef.bridge.driver.launch.azbatch.AzureBatchLauncher;
+import org.apache.reef.bridge.driver.launch.local.LocalLauncher;
+import org.apache.reef.bridge.driver.launch.yarn.YarnLauncher;
+import org.apache.reef.bridge.driver.service.DriverServiceConfigurationProvider;
+import org.apache.reef.bridge.driver.service.grpc.GRPCDriverServiceConfigurationProvider;
+import org.apache.reef.bridge.driver.client.JavaDriverClientLauncher;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.runtime.azbatch.AzureBatchClasspathProvider;
+import org.apache.reef.runtime.common.files.*;
+import org.apache.reef.runtime.common.launch.JavaLaunchCommandBuilder;
+import org.apache.reef.runtime.local.LocalClasspathProvider;
+import org.apache.reef.runtime.yarn.YarnClasspathProvider;
+import org.apache.reef.runtime.yarn.util.YarnConfigurationConstructor;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.util.Collections;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Driver Service Launcher - main class.
+ */
+@ClientSide
+@Unstable
+public final class DriverServiceLauncher {
+
+  /**
+   * Standard Java logger.
+   */
+  private static final Logger LOG = Logger.getLogger(DriverServiceLauncher.class.getName());
+
+  private static final Tang TANG = Tang.Factory.getTang();
+
+  /**
+   * This class should not be instantiated.
+   */
+  private DriverServiceLauncher() {
+    throw new RuntimeException("Do not instantiate this class!");
+  }
+
+  /**
+   * Submit a new REEF driver service (job).
+   * @param driverClientConfigurationProto client configuration protocol buffer
+   * @param driverClientConfiguration driver configuration
+   * @return LauncherStatus
+   * @throws InjectionException
+   * @throws IOException
+   */
+  public static LauncherStatus submit(
+      final ClientProtocol.DriverClientConfiguration driverClientConfigurationProto,
+      final Configuration driverClientConfiguration)
+      throws InjectionException, IOException {
+    ClientProtocol.DriverClientConfiguration.Builder builder =
+        ClientProtocol.DriverClientConfiguration.newBuilder(driverClientConfigurationProto);
+    final File driverClientConfigurationFile = File.createTempFile("driverclient", ".conf");
+    try {
+      // Write driver client configuration to a file
+      final Injector driverClientInjector = TANG.newInjector(driverClientConfiguration);
+      final ConfigurationSerializer configurationSerializer =
+          driverClientInjector.getInstance(ConfigurationSerializer.class);
+      configurationSerializer.toFile(driverClientConfiguration, driverClientConfigurationFile);
+
+      // Resolve Runtime ClassPath Provider.
+      final Configuration runtimeClassPathProvider;
+      switch (driverClientConfigurationProto.getRuntimeCase()) {
+      case YARN_RUNTIME:
+        runtimeClassPathProvider = TANG.newConfigurationBuilder()
+            .bind(RuntimeClasspathProvider.class, YarnClasspathProvider.class)
+            .bindConstructor(org.apache.hadoop.yarn.conf.YarnConfiguration.class,
+                YarnConfigurationConstructor.class)
+            .build();
+        break;
+      case LOCAL_RUNTIME:
+        runtimeClassPathProvider = TANG.newConfigurationBuilder()
+            .bind(RuntimeClasspathProvider.class, LocalClasspathProvider.class)
+            .build();
+        break;
+      case AZBATCH_RUNTIME:
+        runtimeClassPathProvider = TANG.newConfigurationBuilder()
+            .bind(RuntimeClasspathProvider.class, AzureBatchClasspathProvider.class)
+            .build();
+        break;
+      default:
+        throw new RuntimeException("unknown runtime " + driverClientConfigurationProto.getRuntimeCase());
+      }
+      final Injector runtimeInjector = TANG.newInjector(runtimeClassPathProvider);
+      final REEFFileNames fileNames = runtimeInjector.getInstance(REEFFileNames.class);
+      final ClasspathProvider classpathProvider = runtimeInjector.getInstance(ClasspathProvider.class);
+      final List<String> launchCommand = new JavaLaunchCommandBuilder(JavaDriverClientLauncher.class, null)
+          .setConfigurationFilePaths(
+              Collections.singletonList("./" + fileNames.getLocalFolderPath() + "/" +
+                  driverClientConfigurationFile.getName()))
+          .setJavaPath("java")
+          .setClassPath(driverClientConfigurationProto.getOperatingSystem() ==
+              ClientProtocol.DriverClientConfiguration.OS.WINDOWS ?
+              StringUtils.join(classpathProvider.getDriverClasspath(), ";") :
+              StringUtils.join(classpathProvider.getDriverClasspath(), ":"))
+          .build();
+      final String cmd = StringUtils.join(launchCommand, ' ');
+      builder.setDriverClientLaunchCommand(cmd);
+      builder.addLocalFiles(driverClientConfigurationFile.getAbsolutePath());
+
+      return launch(driverClientConfigurationProto);
+    } finally {
+      driverClientConfigurationFile.deleteOnExit();
+    }
+  }
+
+  private static LauncherStatus launch(
+      final ClientProtocol.DriverClientConfiguration driverClientConfigurationProto) throws InjectionException {
+
+    final ClientProtocol.DriverClientConfiguration.RuntimeCase runtime =
+        driverClientConfigurationProto.getRuntimeCase();
+
+    final Class<? extends BridgeDriverLauncher> launcherClass;
+    switch (runtime) {
+    case YARN_RUNTIME:
+      launcherClass = YarnLauncher.class;
+      break;
+    case LOCAL_RUNTIME:
+      launcherClass = LocalLauncher.class;
+      break;
+    case AZBATCH_RUNTIME:
+      launcherClass = AzureBatchLauncher.class;
+      break;
+    default:
+      throw new RuntimeException("Unknown runtime: " + runtime);
+    }
+    final Configuration jobSubmissionClientConfig = TANG.newConfigurationBuilder()
+        .bindImplementation(BridgeDriverLauncher.class, launcherClass)
+        .bindImplementation(DriverServiceConfigurationProvider.class,
+            GRPCDriverServiceConfigurationProvider.class)
+        .build();
+    final BridgeDriverLauncher driverServiceLauncher =
+        TANG.newInjector(jobSubmissionClientConfig).getInstance(launcherClass);
+    return driverServiceLauncher.launch(driverClientConfigurationProto);
+  }
+
+  /**
+   * Main method that launches the REEF job.
+   *
+   * @param args command line parameters.
+   */
+  public static void main(final String[] args) throws IOException, InjectionException {
+    if (args.length != 1) {
+      LOG.log(Level.SEVERE,
+          "Expected a single command line argument with a file containing client protobuf driver configuration");
+      System.exit(1);
+    }
+    final ClientProtocol.DriverClientConfiguration.Builder driverClientConfigurationProtoBuilder =
+        ClientProtocol.DriverClientConfiguration.newBuilder();
+    try (final Reader reader = new FileReader(args[0])) {
+      JsonFormat.parser()
+          .usingTypeRegistry(JsonFormat.TypeRegistry.getEmptyTypeRegistry())
+          .merge(reader, driverClientConfigurationProtoBuilder);
+    }
+    final ClientProtocol.DriverClientConfiguration driverClientConfigurationProto =
+        driverClientConfigurationProtoBuilder.build();
+    final LauncherStatus status = launch(driverClientConfigurationProto);
+    LOG.log(Level.INFO, "Status: {0}", status);
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java
index 9967d9b..9b6021d 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Java client launcher.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.client;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/AlarmDispatchHandler.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/AlarmDispatchHandler.java
index 9967d9b..e93a5e7 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/AlarmDispatchHandler.java
@@ -16,20 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.wake.EventHandler;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Alarm dispatch handler.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Private
+@DefaultImplementation(DriverClientClock.class)
+public interface AlarmDispatchHandler extends EventHandler<String> {
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DefaultDriverClientStopHandler.java
similarity index 54%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DefaultDriverClientStopHandler.java
index 9967d9b..f04e9f3 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DefaultDriverClientStopHandler.java
@@ -16,20 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.event.StopTime;
+
+import javax.inject.Inject;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Default java client driver stop handler.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Private
+public final class DefaultDriverClientStopHandler implements EventHandler<StopTime> {
+
+  private static final Logger LOG = Logger.getLogger(DefaultDriverClientStopHandler.class.getName());
+
+  @Inject
+  private DefaultDriverClientStopHandler() {}
+
+  @Override
+  public void onNext(final StopTime value) {
+    LOG.log(Level.FINEST, "Stop time {0}", value);
+  }
 }
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientClock.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientClock.java
new file mode 100644
index 0000000..df16b72
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientClock.java
@@ -0,0 +1,132 @@
+/*
+ * 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.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.Clock;
+import org.apache.reef.wake.time.Time;
+import org.apache.reef.wake.time.event.Alarm;
+import org.apache.reef.wake.time.runtime.Timer;
+import org.apache.reef.wake.time.runtime.event.ClientAlarm;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * The bridge driver client clock.
+ */
+@Private
+public final class DriverClientClock implements Clock, AlarmDispatchHandler {
+
+  private static final Logger LOG = Logger.getLogger(DriverClientClock.class.getName());
+
+  private final DriverClientService driverClientService;
+
+  private final DriverServiceClient driverServiceClient;
+
+  private final Timer timer;
+
+  private final Map<String, ClientAlarm> alarmMap = new HashMap<>();
+
+  private boolean closed = false;
+
+  @Inject
+  private DriverClientClock(
+      final Timer timer,
+      final DriverClientService driverClientService,
+      final DriverServiceClient driverServiceClient) {
+    this.timer = timer;
+    this.driverClientService = driverClientService;
+    this.driverServiceClient = driverServiceClient;
+  }
+
+  @Override
+  public Time scheduleAlarm(final int offset, final EventHandler<Alarm> handler) {
+    LOG.log(Level.INFO, "Schedule alarm offset {0}", offset);
+    final ClientAlarm alarm = new ClientAlarm(this.timer.getCurrent() + offset, handler);
+    final String alarmId = UUID.randomUUID().toString();
+    this.alarmMap.put(alarmId, alarm);
+    this.driverServiceClient.onSetAlarm(alarmId, offset);
+    LOG.log(Level.INFO, "Alarm {0} scheduled at offset {1}", new Object[]{alarmId, offset});
+    return alarm;
+  }
+
+  @Override
+  public void close() {
+    stop();
+  }
+
+  @Override
+  public void stop() {
+    if (!closed) {
+      this.closed = true;
+      this.driverServiceClient.onShutdown();
+    }
+  }
+
+  @Override
+  public void stop(final Throwable exception) {
+    if (!closed) {
+      this.closed = true;
+      this.driverServiceClient.onShutdown(exception);
+    }
+  }
+
+  @Override
+  public boolean isIdle() {
+    return this.closed || this.alarmMap.isEmpty();
+  }
+
+  @Override
+  public boolean isClosed() {
+    return this.closed;
+  }
+
+  @Override
+  public void run() {
+    try {
+      this.driverClientService.start();
+      this.driverClientService.awaitTermination();
+    } catch (final IOException | InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Alarm clock event handler.
+   * @param alarmId alarm identifier
+   */
+  @Override
+  public void onNext(final String alarmId) {
+    LOG.log(Level.INFO, "Alarm {0} triggered", alarmId);
+    final ClientAlarm clientAlarm = this.alarmMap.remove(alarmId);
+    if (clientAlarm != null) {
+      clientAlarm.run();
+    } else {
+      LOG.log(Level.SEVERE, "Unknown alarm id {0}", alarmId);
+    }
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientConfiguration.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientConfiguration.java
new file mode 100644
index 0000000..4df8941
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientConfiguration.java
@@ -0,0 +1,204 @@
+/*
+ * 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.bridge.driver.client;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.bridge.driver.client.parameters.DriverClientDispatchThreadCount;
+import org.apache.reef.bridge.driver.client.parameters.ClientDriverStopHandler;
+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.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;
+
+/**
+ * Driver client configuration.
+ */
+@Unstable
+public final class DriverClientConfiguration 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<>();
+
+  // ***** CLIENT HANDLER BINDINGS:
+
+  /**
+   * Event handler for client messages. Defaults to logging if not bound.
+   */
+  public static final OptionalImpl<EventHandler<byte[]>> ON_CLIENT_MESSAGE = new OptionalImpl<>();
+
+  /**
+   * Event handler for close messages sent by the client. Defaults to job failure if not bound.
+   */
+  public static final OptionalImpl<EventHandler<Void>> ON_CLIENT_CLOSED = new OptionalImpl<>();
+
+  /**
+   * Event handler for close messages sent by the client. Defaults to job failure if not bound.
+   */
+  public static final OptionalImpl<EventHandler<byte[]>> ON_CLIENT_CLOSED_MESSAGE = 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<>();
+
+  /**
+   * Number of dispatch threads to use.
+   */
+  public static final OptionalImpl<Integer> CLIENT_DRIVER_DISPATCH_THREAD_COUNT = new OptionalImpl<>();
+
+  /**
+   * Alarm dispatch handler.
+   */
+  public static final OptionalImpl<AlarmDispatchHandler> ALARM_DISPATCH_HANDLER = new OptionalImpl<>();
+
+  /**
+   * Default to gRPC Driver Client Service.
+   */
+  public static final OptionalImpl<DriverClientService> DRIVER_CLIENT_SERVICE = new OptionalImpl<>();
+
+  /**
+   * Default to gRPC Driver Service Client.
+   */
+  public static final OptionalImpl<DriverServiceClient> DRIVER_SERVICE_CLIENT = new OptionalImpl<>();
+
+  /**
+   * ConfigurationModule to fill out to get a legal Driver Configuration.
+   */
+  public static final ConfigurationModule CONF = new DriverClientConfiguration()
+      .bindImplementation(Clock.class, DriverClientClock.class)
+      .bindImplementation(EvaluatorRequestor.class, DriverClientEvaluatorRequestor.class)
+      .bindImplementation(AlarmDispatchHandler.class, ALARM_DISPATCH_HANDLER)
+      .bindImplementation(DriverClientService.class, DRIVER_CLIENT_SERVICE)
+      .bindImplementation(DriverServiceClient.class, DRIVER_SERVICE_CLIENT)
+
+      .bindNamedParameter(DriverClientDispatchThreadCount.class, CLIENT_DRIVER_DISPATCH_THREAD_COUNT)
+
+      // Driver start/stop handlers
+      .bindSetEntry(DriverStartHandler.class, ON_DRIVER_STARTED)
+      .bindSetEntry(ClientDriverStopHandler.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)
+
+      // Client handlers
+      .bindSetEntry(ClientMessageHandlers.class, ON_CLIENT_MESSAGE)
+      .bindSetEntry(ClientCloseHandlers.class, ON_CLIENT_CLOSED)
+      .bindSetEntry(ClientCloseWithMessageHandlers.class, ON_CLIENT_CLOSED_MESSAGE)
+
+      .build();
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientDispatcher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientDispatcher.java
new file mode 100644
index 0000000..35285f4
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientDispatcher.java
@@ -0,0 +1,364 @@
+/*
+ * 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.bridge.driver.client;
+
+import com.google.common.collect.Sets;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.parameters.DriverClientDispatchThreadCount;
+import org.apache.reef.bridge.driver.client.parameters.ClientDriverStopHandler;
+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.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.runtime.common.utils.DispatchingEStage;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+
+import javax.inject.Inject;
+import java.util.Set;
+
+/**
+ * Async dispatch of client driver events.
+ */
+@Private
+public final class DriverClientDispatcher {
+
+  /**
+   * Dispatcher used for application provided event handlers.
+   */
+  private final DispatchingEStage applicationDispatcher;
+
+  /**
+   * Dispatcher for client close events.
+   */
+  private final DispatchingEStage clientCloseDispatcher;
+
+  /**
+   * Dispatcher for client close with message events.
+   */
+  private final DispatchingEStage clientCloseWithMessageDispatcher;
+
+  /**
+   * Dispatcher for client messages.
+   */
+  private final DispatchingEStage clientMessageDispatcher;
+
+  /**
+   * The alarm dispatcher.
+   */
+  private final DispatchingEStage alarmDispatcher;
+
+  /**
+   * Driver restart dispatcher.
+   */
+  private final DispatchingEStage driverRestartDispatcher;
+
+  /**
+   * Synchronous set of stop handlers.
+   */
+  private final Set<EventHandler<StopTime>> stopHandlers;
+
+  @Inject
+  private DriverClientDispatcher(
+      final DriverClientExceptionHandler driverExceptionHandler,
+      final AlarmDispatchHandler alarmDispatchHandler,
+      @Parameter(DriverClientDispatchThreadCount.class)
+      final Integer numberOfThreads,
+      // Application-provided start and stop handlers
+      @Parameter(DriverStartHandler.class)
+      final Set<EventHandler<StartTime>> startHandlers,
+      @Parameter(ClientDriverStopHandler.class)
+      final Set<EventHandler<StopTime>> stopHandlers,
+      // Application-provided Context event handlers
+      @Parameter(ContextActiveHandlers.class)
+      final Set<EventHandler<ActiveContext>> contextActiveHandlers,
+      @Parameter(ContextClosedHandlers.class)
+      final Set<EventHandler<ClosedContext>> contextClosedHandlers,
+      @Parameter(ContextFailedHandlers.class)
+      final Set<EventHandler<FailedContext>> contextFailedHandlers,
+      @Parameter(ContextMessageHandlers.class)
+      final Set<EventHandler<ContextMessage>> contextMessageHandlers,
+      // Application-provided Task event handlers
+      @Parameter(TaskRunningHandlers.class)
+      final Set<EventHandler<RunningTask>> taskRunningHandlers,
+      @Parameter(TaskCompletedHandlers.class)
+      final Set<EventHandler<CompletedTask>> taskCompletedHandlers,
+      @Parameter(TaskSuspendedHandlers.class)
+      final Set<EventHandler<SuspendedTask>> taskSuspendedHandlers,
+      @Parameter(TaskMessageHandlers.class)
+      final Set<EventHandler<TaskMessage>> taskMessageEventHandlers,
+      @Parameter(TaskFailedHandlers.class)
+      final Set<EventHandler<FailedTask>> taskExceptionEventHandlers,
+      // Application-provided Evaluator event handlers
+      @Parameter(EvaluatorAllocatedHandlers.class)
+      final Set<EventHandler<AllocatedEvaluator>> evaluatorAllocatedHandlers,
+      @Parameter(EvaluatorFailedHandlers.class)
+      final Set<EventHandler<FailedEvaluator>> evaluatorFailedHandlers,
+      @Parameter(EvaluatorCompletedHandlers.class)
+      final Set<EventHandler<CompletedEvaluator>> evaluatorCompletedHandlers,
+      // Client handlers
+      @Parameter(ClientCloseHandlers.class)
+      final Set<EventHandler<Void>> clientCloseHandlers,
+      @Parameter(ClientCloseWithMessageHandlers.class)
+      final Set<EventHandler<byte[]>> clientCloseWithMessageHandlers,
+      @Parameter(ClientMessageHandlers.class)
+      final Set<EventHandler<byte[]>> clientMessageHandlers) {
+    this.applicationDispatcher = new DispatchingEStage(
+        driverExceptionHandler, numberOfThreads, "ClientDriverDispatcher");
+    // Application start and stop handlers
+    this.applicationDispatcher.register(StartTime.class, startHandlers);
+    this.stopHandlers = stopHandlers; // must be called synchronously
+    // Application Context event handlers
+    this.applicationDispatcher.register(ActiveContext.class, contextActiveHandlers);
+    this.applicationDispatcher.register(ClosedContext.class, contextClosedHandlers);
+    this.applicationDispatcher.register(FailedContext.class, contextFailedHandlers);
+    this.applicationDispatcher.register(ContextMessage.class, contextMessageHandlers);
+
+    // Application Task event handlers.
+    this.applicationDispatcher.register(RunningTask.class, taskRunningHandlers);
+    this.applicationDispatcher.register(CompletedTask.class, taskCompletedHandlers);
+    this.applicationDispatcher.register(SuspendedTask.class, taskSuspendedHandlers);
+    this.applicationDispatcher.register(TaskMessage.class, taskMessageEventHandlers);
+    this.applicationDispatcher.register(FailedTask.class, taskExceptionEventHandlers);
+
+    // Application Evaluator event handlers
+    this.applicationDispatcher.register(AllocatedEvaluator.class, evaluatorAllocatedHandlers);
+    this.applicationDispatcher.register(CompletedEvaluator.class, evaluatorCompletedHandlers);
+    this.applicationDispatcher.register(FailedEvaluator.class, evaluatorFailedHandlers);
+
+    // Client event handlers;
+    this.clientCloseDispatcher = new DispatchingEStage(this.applicationDispatcher);
+    this.clientCloseDispatcher.register(Void.class, clientCloseHandlers);
+
+    this.clientCloseWithMessageDispatcher = new DispatchingEStage(this.applicationDispatcher);
+    this.clientCloseWithMessageDispatcher.register(byte[].class, clientCloseWithMessageHandlers);
+
+    this.clientMessageDispatcher = new DispatchingEStage(this.applicationDispatcher);
+    this.clientMessageDispatcher.register(byte[].class, clientMessageHandlers);
+
+    // Alarm event handlers
+    this.alarmDispatcher = new DispatchingEStage(this.applicationDispatcher);
+    this.alarmDispatcher.register(String.class,
+        Sets.newHashSet((EventHandler<String>)alarmDispatchHandler));
+
+    // Driver restart dispatcher
+    this.driverRestartDispatcher = new DispatchingEStage(this.applicationDispatcher);
+  }
+
+  @Inject
+  private DriverClientDispatcher(
+      final DriverClientExceptionHandler driverExceptionHandler,
+      final AlarmDispatchHandler alarmDispatchHandler,
+      @Parameter(DriverClientDispatchThreadCount.class)
+      final Integer numberOfThreads,
+      // Application-provided start and stop handlers
+      @Parameter(DriverStartHandler.class)
+      final Set<EventHandler<StartTime>> startHandlers,
+      @Parameter(ClientDriverStopHandler.class)
+      final Set<EventHandler<StopTime>> stopHandlers,
+      // Application-provided Context event handlers
+      @Parameter(ContextActiveHandlers.class)
+      final Set<EventHandler<ActiveContext>> contextActiveHandlers,
+      @Parameter(ContextClosedHandlers.class)
+      final Set<EventHandler<ClosedContext>> contextClosedHandlers,
+      @Parameter(ContextFailedHandlers.class)
+      final Set<EventHandler<FailedContext>> contextFailedHandlers,
+      @Parameter(ContextMessageHandlers.class)
+      final Set<EventHandler<ContextMessage>> contextMessageHandlers,
+      // Application-provided Task event handlers
+      @Parameter(TaskRunningHandlers.class)
+      final Set<EventHandler<RunningTask>> taskRunningHandlers,
+      @Parameter(TaskCompletedHandlers.class)
+      final Set<EventHandler<CompletedTask>> taskCompletedHandlers,
+      @Parameter(TaskSuspendedHandlers.class)
+      final Set<EventHandler<SuspendedTask>> taskSuspendedHandlers,
+      @Parameter(TaskMessageHandlers.class)
+      final Set<EventHandler<TaskMessage>> taskMessageEventHandlers,
+      @Parameter(TaskFailedHandlers.class)
+      final Set<EventHandler<FailedTask>> taskExceptionEventHandlers,
+      // Application-provided Evaluator event handlers
+      @Parameter(EvaluatorAllocatedHandlers.class)
+      final Set<EventHandler<AllocatedEvaluator>> evaluatorAllocatedHandlers,
+      @Parameter(EvaluatorFailedHandlers.class)
+      final Set<EventHandler<FailedEvaluator>> evaluatorFailedHandlers,
+      @Parameter(EvaluatorCompletedHandlers.class)
+      final Set<EventHandler<CompletedEvaluator>> evaluatorCompletedHandlers,
+      // Client handlers
+      @Parameter(ClientCloseHandlers.class)
+      final Set<EventHandler<Void>> clientCloseHandlers,
+      @Parameter(ClientCloseWithMessageHandlers.class)
+      final Set<EventHandler<byte[]>> clientCloseWithMessageHandlers,
+      @Parameter(ClientMessageHandlers.class)
+      final Set<EventHandler<byte[]>> clientMessageHandlers,
+      // Driver restart handlers
+      @Parameter(DriverRestartHandler.class)
+      final Set<EventHandler<DriverRestarted>> driverRestartHandlers,
+      @Parameter(DriverRestartTaskRunningHandlers.class)
+      final Set<EventHandler<RunningTask>> driverRestartTaskRunningHandlers,
+      @Parameter(DriverRestartContextActiveHandlers.class)
+      final Set<EventHandler<ActiveContext>> driverRestartActiveContextHandlers,
+      @Parameter(DriverRestartCompletedHandlers.class)
+      final Set<EventHandler<DriverRestartCompleted>> driverRestartCompletedHandlers,
+      @Parameter(DriverRestartFailedEvaluatorHandlers.class)
+      final Set<EventHandler<FailedEvaluator>> driverRestartFailedEvaluatorHandlers) {
+    this(
+        driverExceptionHandler,
+        alarmDispatchHandler,
+        numberOfThreads,
+        startHandlers,
+        stopHandlers,
+        contextActiveHandlers,
+        contextClosedHandlers,
+        contextFailedHandlers,
+        contextMessageHandlers,
+        taskRunningHandlers,
+        taskCompletedHandlers,
+        taskSuspendedHandlers,
+        taskMessageEventHandlers,
+        taskExceptionEventHandlers,
+        evaluatorAllocatedHandlers,
+        evaluatorFailedHandlers,
+        evaluatorCompletedHandlers,
+        clientCloseHandlers,
+        clientCloseWithMessageHandlers,
+        clientMessageHandlers);
+    // Register driver restart handlers.
+    this.driverRestartDispatcher.register(DriverRestarted.class, driverRestartHandlers);
+    this.driverRestartDispatcher.register(RunningTask.class, driverRestartTaskRunningHandlers);
+    this.driverRestartDispatcher.register(ActiveContext.class, driverRestartActiveContextHandlers);
+    this.driverRestartDispatcher.register(DriverRestartCompleted.class, driverRestartCompletedHandlers);
+    this.driverRestartDispatcher.register(FailedEvaluator.class, driverRestartFailedEvaluatorHandlers);
+  }
+
+  public void dispatchRestart(final DriverRestarted driverRestarted) {
+    this.driverRestartDispatcher.onNext(DriverRestarted.class, driverRestarted);
+  }
+
+  public void dispatchRestart(final RunningTask task) {
+    this.driverRestartDispatcher.onNext(RunningTask.class, task);
+  }
+
+  public void dispatchRestart(final ActiveContext context) {
+    this.driverRestartDispatcher.onNext(ActiveContext.class, context);
+  }
+
+  public void dispatchRestart(final DriverRestartCompleted completed) {
+    this.driverRestartDispatcher.onNext(DriverRestartCompleted.class, completed);
+  }
+
+  public void dispatchRestart(final FailedEvaluator evaluator) {
+    this.driverRestartDispatcher.onNext(FailedEvaluator.class, evaluator);
+  }
+
+  public void dispatch(final StartTime startTime) {
+    this.applicationDispatcher.onNext(StartTime.class, startTime);
+  }
+
+  /**
+   * We must implement this synchronously in order to catch exceptions and
+   * forward them back via the bridge before the server shuts down, after
+   * this method returns.
+   * @param stopTime stop time
+   */
+  @SuppressWarnings("checkstyle:illegalCatch")
+  public Throwable dispatch(final StopTime stopTime) {
+    try {
+      for (final EventHandler<StopTime> handler : stopHandlers) {
+        handler.onNext(stopTime);
+      }
+      return null;
+    } catch (Throwable t) {
+      return t;
+    }
+  }
+
+  public void dispatch(final ActiveContext context) {
+    this.applicationDispatcher.onNext(ActiveContext.class, context);
+  }
+
+  public void dispatch(final ClosedContext context) {
+    this.applicationDispatcher.onNext(ClosedContext.class, context);
+  }
+
+  public void dispatch(final FailedContext context) {
+    this.applicationDispatcher.onNext(FailedContext.class, context);
+  }
+
+  public void dispatch(final ContextMessage message) {
+    this.applicationDispatcher.onNext(ContextMessage.class, message);
+  }
+
+  public void dispatch(final AllocatedEvaluator evaluator) {
+    this.applicationDispatcher.onNext(AllocatedEvaluator.class, evaluator);
+  }
+
+  public void dispatch(final FailedEvaluator evaluator) {
+    this.applicationDispatcher.onNext(FailedEvaluator.class, evaluator);
+  }
+
+  public void dispatch(final CompletedEvaluator evaluator) {
+    this.applicationDispatcher.onNext(CompletedEvaluator.class, evaluator);
+  }
+
+  public void dispatch(final RunningTask task) {
+    this.applicationDispatcher.onNext(RunningTask.class, task);
+  }
+
+  public void dispatch(final CompletedTask task) {
+    this.applicationDispatcher.onNext(CompletedTask.class, task);
+  }
+
+  public void dispatch(final FailedTask task) {
+    this.applicationDispatcher.onNext(FailedTask.class, task);
+  }
+
+  public void dispatch(final SuspendedTask task) {
+    this.applicationDispatcher.onNext(SuspendedTask.class, task);
+  }
+
+  public void dispatch(final TaskMessage message) {
+    this.applicationDispatcher.onNext(TaskMessage.class, message);
+  }
+
+  public void clientCloseDispatch() {
+    this.clientCloseDispatcher.onNext(Void.class, null);
+  }
+
+  public void clientCloseWithMessageDispatch(final byte[] message) {
+    this.clientCloseWithMessageDispatcher.onNext(byte[].class, message);
+  }
+
+  public void clientMessageDispatch(final byte[] message) {
+    this.clientMessageDispatcher.onNext(byte[].class, message);
+  }
+
+  public void dispatchAlarm(final String alarmId) {
+    this.alarmDispatcher.onNext(String.class, alarmId);
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientEvaluatorRequestor.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientEvaluatorRequestor.java
new file mode 100644
index 0000000..75208f5
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientEvaluatorRequestor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.EvaluatorRequest;
+import org.apache.reef.driver.evaluator.EvaluatorRequestor;
+
+import javax.inject.Inject;
+
+/**
+ * Driver Client evaluator requestor.
+ */
+@Private
+public final class DriverClientEvaluatorRequestor implements EvaluatorRequestor {
+
+  private final DriverServiceClient driverServiceClient;
+
+  private final DriverClientService driverClientService;
+
+  @Inject
+  private DriverClientEvaluatorRequestor(
+      final DriverServiceClient driverServiceClient,
+      final DriverClientService driverClientService) {
+    this.driverServiceClient = driverServiceClient;
+    this.driverClientService = driverClientService;
+  }
+
+  @Override
+  public void submit(final EvaluatorRequest req) {
+    this.driverClientService.notifyEvaluatorRequest(req.getNumber());
+    this.driverServiceClient.onEvaluatorRequest(req);
+  }
+
+  @Override
+  public EvaluatorRequest.Builder newRequest() {
+    return new DriverClientEvaluatorRequestor.Builder();
+  }
+
+  /**
+   * {@link DriverClientEvaluatorRequestor.Builder} extended with a new submit method.
+   * {@link EvaluatorRequest}s are built using this builder.
+   */
+  public final class Builder extends EvaluatorRequest.Builder<DriverClientEvaluatorRequestor.Builder> {
+    @Override
+    public void submit() {
+      DriverClientEvaluatorRequestor.this.submit(this.build());
+    }
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientExceptionHandler.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientExceptionHandler.java
new file mode 100644
index 0000000..363694d
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientExceptionHandler.java
@@ -0,0 +1,50 @@
+/*
+ * 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.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.Clock;
+
+import javax.inject.Inject;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Driver client exception handler.
+ */
+@Private
+public final class DriverClientExceptionHandler implements EventHandler<Throwable> {
+  private static final Logger LOG = Logger.getLogger(DriverClientExceptionHandler.class.getName());
+
+  private final Clock clock;
+
+  @Inject
+  private DriverClientExceptionHandler(final Clock clock) {
+    LOG.log(Level.FINE, "Instantiated 'DriverExceptionHandler'");
+    this.clock = clock;
+  }
+
+  @Override
+  public void onNext(final Throwable throwable) {
+    LOG.log(Level.SEVERE, "Exception in DriverClient - stopping", throwable);
+    this.clock.stop(throwable);
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientService.java
similarity index 54%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientService.java
index 9967d9b..42b00af 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverClientService.java
@@ -16,20 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+
+import java.io.IOException;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Interface that driver client services implement.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Private
+public interface DriverClientService {
+
+  /**
+   * Start the DriverClient service.
+   * @throws IOException when unable to start service
+   */
+  void start() throws IOException;
+
+  /**
+   * Notify that the count number of evaluators have been
+   * requested by the application.
+   * @param count of the number of evaluators
+   */
+  void notifyEvaluatorRequest(final int count);
+
+  /**
+   * Wait for termination of driver client service.
+   */
+  void awaitTermination() throws InterruptedException;
 }
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverServiceClient.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverServiceClient.java
new file mode 100644
index 0000000..038a529
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/DriverServiceClient.java
@@ -0,0 +1,140 @@
+/*
+ * 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.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.EvaluatorRequest;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.util.Optional;
+
+import java.io.File;
+import java.util.List;
+
+/**
+ * Forwards application requests to driver server.
+ */
+@Private
+public interface DriverServiceClient {
+
+  void onInitializationException(final Throwable ex);
+
+  /**
+   * Initiate shutdown.
+   */
+  void onShutdown();
+
+  /**
+   * Initiate shutdown with error.
+   * @param ex exception error
+   */
+  void onShutdown(final Throwable ex);
+
+  /**
+   * Set alarm.
+   * @param alarmId alarm identifier
+   * @param timeoutMS timeout in milliseconds
+   */
+  void onSetAlarm(final String alarmId, final int timeoutMS);
+
+  /**
+   * Request evaluators.
+   * @param evaluatorRequest event
+   */
+  void onEvaluatorRequest(final EvaluatorRequest evaluatorRequest);
+
+  /**
+   * Close evaluator.
+   * @param evalautorId to close
+   */
+  void onEvaluatorClose(final String evalautorId);
+
+  /**
+   * Submit context and/or task.
+   * @param evaluatorId to submit against
+   * @param contextConfiguration context configuration
+   * @param taskConfiguration task configuration
+   * @param evaluatorProcess evaluator process
+   * @param addFileList to include
+   * @param addLibraryList to include
+   */
+  void onEvaluatorSubmit(
+      final String evaluatorId,
+      final Optional<Configuration> contextConfiguration,
+      final Optional<Configuration> taskConfiguration,
+      final Optional<JVMClientProcess> evaluatorProcess,
+      final List<File> addFileList,
+      final List<File> addLibraryList);
+
+  // Context Operations
+
+  /**
+   * Close context.
+   * @param contextId to close
+   */
+  void onContextClose(final String contextId);
+
+  /**
+   * Submit child context.
+   * @param contextId to submit against
+   * @param contextConfiguration for child context
+   */
+  void onContextSubmitContext(
+      final String contextId,
+      final Configuration contextConfiguration);
+
+  /**
+   * Submit task.
+   * @param contextId to submit against
+   * @param taskConfiguration for task
+   */
+  void onContextSubmitTask(
+      final String contextId,
+      final Configuration taskConfiguration);
+
+  /**
+   * Send message to context.
+   * @param contextId to destination context
+   * @param message to send
+   */
+  void onContextMessage(final String contextId, final byte[] message);
+
+  // Task operations
+
+  /**
+   * Close the task.
+   * @param taskId to close
+   * @param message optional message to include
+   */
+  void onTaskClose(final String taskId, final Optional<byte[]> message);
+
+  /**
+   * Send task a message.
+   * @param taskId of destination task
+   * @param message to send
+   */
+  void onTaskMessage(final String taskId, final byte[] message);
+
+  /**
+   * Suspend a running task.
+   * @param taskId task identifier
+   * @param message optional message
+   */
+  void onSuspendTask(final String taskId, final Optional<byte[]> message);
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/JVMClientProcess.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/JVMClientProcess.java
new file mode 100644
index 0000000..809ea42
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/JVMClientProcess.java
@@ -0,0 +1,120 @@
+/*
+ * 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.bridge.driver.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.EvaluatorProcess;
+import org.apache.reef.driver.evaluator.EvaluatorType;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Stub class for Client Process.
+ */
+@Private
+public final class JVMClientProcess implements EvaluatorProcess {
+
+  private boolean optionSet = false;
+
+  private int megaBytes = 0;
+
+  private String configurationFileName = null;
+
+  private String standardOut = null;
+
+  private String standardErr = null;
+
+  private final List<String> optionList = new ArrayList<>();
+
+  public JVMClientProcess() {
+  }
+
+  @Override
+  public List<String> getCommandLine() {
+    throw new UnsupportedOperationException("Getting command line on the client is not supported.");
+  }
+
+  @Override
+  public EvaluatorType getType() {
+    return EvaluatorType.JVM;
+  }
+
+  @Override
+  public JVMClientProcess setMemory(final int mb) {
+    this.megaBytes = mb;
+    this.optionSet = true;
+    return this;
+  }
+
+  public int getMemory() {
+    return this.megaBytes;
+  }
+
+  @Override
+  public boolean isOptionSet() {
+    return optionSet;
+  }
+
+  @Override
+  public JVMClientProcess setConfigurationFileName(final String configurationFileName) {
+    this.configurationFileName = configurationFileName;
+    return this;
+  }
+
+  public String getConfigurationFileName() {
+    return this.configurationFileName;
+  }
+
+  @Override
+  public JVMClientProcess setStandardOut(final String standardOut) {
+    this.standardOut = standardOut;
+    return this;
+  }
+
+  public String getStandardOut() {
+    return this.standardOut;
+  }
+
+  @Override
+  public JVMClientProcess setStandardErr(final String standardErr) {
+    this.standardErr = standardErr;
+    return this;
+  }
+
+  public String getStandardErr() {
+    return this.standardErr;
+  }
+
+  /**
+   * Add a JVM option.
+   * @param option The full option, e.g. "-XX:+PrintGCDetails", "-Xms500m"
+   * @return this
+   */
+  public JVMClientProcess addOption(final String option) {
+    this.optionList.add(option);
+    optionSet = true;
+    return this;
+  }
+
+  public List<String> getOptions() {
+    return this.optionList;
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/JavaDriverClientLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/JavaDriverClientLauncher.java
new file mode 100644
index 0000000..7522cba
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/JavaDriverClientLauncher.java
@@ -0,0 +1,227 @@
+/*
+ * 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.bridge.driver.client;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.bridge.driver.client.grpc.DriverClientGrpcConfiguration;
+import org.apache.reef.bridge.driver.client.grpc.parameters.DriverServicePort;
+import org.apache.reef.runtime.common.REEFLauncher;
+import org.apache.reef.runtime.common.evaluator.PIDStoreStartHandler;
+import org.apache.reef.runtime.common.launch.REEFErrorHandler;
+import org.apache.reef.runtime.common.launch.REEFMessageCodec;
+import org.apache.reef.runtime.common.launch.REEFUncaughtExceptionHandler;
+import org.apache.reef.runtime.common.launch.parameters.ClockConfigurationPath;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.util.EnvironmentUtils;
+import org.apache.reef.util.ThreadLogger;
+import org.apache.reef.util.logging.LoggingSetup;
+import org.apache.reef.wake.remote.RemoteConfiguration;
+import org.apache.reef.wake.time.Clock;
+
+import javax.inject.Inject;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Driver client launcher.
+ */
+@Unstable
+public final class JavaDriverClientLauncher {
+
+  private static final Logger LOG = Logger.getLogger(REEFLauncher.class.getName());
+
+  private static final Tang TANG = Tang.Factory.getTang();
+
+  private static final Configuration LAUNCHER_STATIC_CONFIG =
+      TANG.newConfigurationBuilder()
+          .bindNamedParameter(RemoteConfiguration.ManagerName.class, "DRIVER_CLIENT_LAUNCHER")
+          .bindNamedParameter(RemoteConfiguration.ErrorHandler.class, REEFErrorHandler.class)
+          .bindNamedParameter(RemoteConfiguration.MessageCodec.class, REEFMessageCodec.class)
+          .bindSetEntry(Clock.RuntimeStartHandler.class, PIDStoreStartHandler.class)
+          .build();
+
+  static {
+    LoggingSetup.setupCommonsLogging();
+  }
+
+  /**
+   * Main configuration object of the REEF component we are launching here.
+   */
+  private final Configuration envConfig;
+
+  /**
+   * REEFLauncher is instantiated in the main() method below using
+   * Tang configuration file provided as a command line argument.
+   * @param configurationPath Path to the serialized Tang configuration file.
+   * (The file must be in the local file system).
+   * @param configurationSerializer Serializer used to read the configuration file.
+   * We currently use Avro to serialize Tang configs.
+   */
+  @Inject
+  private JavaDriverClientLauncher(
+      @Parameter(DriverServicePort.class) final Integer driverServicePort,
+      @Parameter(ClockConfigurationPath.class) final String configurationPath,
+      final ConfigurationSerializer configurationSerializer) {
+
+    this.envConfig = Configurations.merge(
+        LAUNCHER_STATIC_CONFIG,
+        DriverClientGrpcConfiguration.CONF
+            .set(DriverClientGrpcConfiguration.DRIVER_SERVICE_PORT, driverServicePort)
+            .build(),
+        readConfigurationFromDisk(configurationPath, configurationSerializer));
+  }
+
+  /**
+   * Instantiate REEF DriverServiceLauncher. This method is called from REEFLauncher.main().
+   * @param clockConfigPath Path to the local file that contains serialized configuration
+   *                        for the driver client.
+   * @return An instance of the configured REEFLauncher object.
+   */
+  private static JavaDriverClientLauncher getLauncher(final String clockConfigPath, final int driverServicePort) {
+    try {
+      final Configuration clockArgConfig = Configurations.merge(
+          LAUNCHER_STATIC_CONFIG,
+          DriverClientGrpcConfiguration.CONF
+              .set(DriverClientGrpcConfiguration.DRIVER_SERVICE_PORT, driverServicePort)
+              .build(),
+          TANG.newConfigurationBuilder()
+              .bindNamedParameter(ClockConfigurationPath.class, clockConfigPath)
+              .build());
+
+      return TANG.newInjector(clockArgConfig).getInstance(JavaDriverClientLauncher.class);
+    } catch (final InjectionException ex) {
+      throw fatal("Unable to instantiate REEFLauncher.", ex);
+    }
+  }
+
+  /**
+   * Read configuration from a given file and deserialize it
+   * into Tang configuration object that can be used for injection.
+   * Configuration is currently serialized using Avro.
+   * This method also prints full deserialized configuration into log.
+   * @param configPath Path to the local file that contains serialized configuration
+   * of a REEF component to launch (can be either Driver or Evaluator).
+   * @param serializer An object to deserialize the configuration file.
+   * @return Tang configuration read and deserialized from a given file.
+   */
+  private static Configuration readConfigurationFromDisk(
+      final String configPath, final ConfigurationSerializer serializer) {
+
+    LOG.log(Level.FINER, "Loading configuration file: {0}", configPath);
+
+    final File evaluatorConfigFile = new File(configPath);
+
+    if (!evaluatorConfigFile.exists()) {
+      throw fatal(
+          "Configuration file " + configPath + " does not exist. Can be an issue in job submission.",
+          new FileNotFoundException(configPath));
+    }
+
+    if (!evaluatorConfigFile.canRead()) {
+      throw fatal(
+          "Configuration file " + configPath + " exists, but can't be read.",
+          new IOException(configPath));
+    }
+
+    try {
+
+      final Configuration config = serializer.fromFile(evaluatorConfigFile);
+      LOG.log(Level.FINEST, "The configuration file loaded: {0}", configPath);
+
+      return config;
+
+    } catch (final IOException e) {
+      throw fatal("Unable to parse the configuration file: " + configPath, e);
+    }
+  }
+
+  /**
+   * Launches a REEF client process (Driver or Evaluator).
+   * @param args Command-line arguments.
+   * Must be a single element containing local path to the configuration file.
+   */
+  @SuppressWarnings("checkstyle:illegalcatch")
+  public static void main(final String[] args) {
+
+    LOG.log(Level.INFO, "Entering JavaDriverClientLauncher.main().");
+
+    LOG.log(Level.FINE, "JavaDriverClientLauncher started with user name [{0}]", System.getProperty("user.name"));
+    LOG.log(Level.FINE, "JavaDriverClientLauncher started. Assertions are {0} in this process.",
+        EnvironmentUtils.areAssertionsEnabled() ? "ENABLED" : "DISABLED");
+
+    if (args.length != 2) {
+      final String message = "JavaDriverClientLauncher have two and only two arguments to specify the runtime clock " +
+          "configuration path and driver service port";
+
+      throw fatal(message, new IllegalArgumentException(message));
+    }
+
+    final JavaDriverClientLauncher launcher = getLauncher(args[0], Integer.parseInt(args[1]));
+
+    Thread.setDefaultUncaughtExceptionHandler(new REEFUncaughtExceptionHandler(launcher.envConfig));
+    final Injector injector = TANG.newInjector(launcher.envConfig);
+    try {
+      final DriverServiceClient driverServiceClient = injector.getInstance(DriverServiceClient.class);
+      try (final Clock reef = injector.getInstance(Clock.class)) {
+        reef.run();
+      } catch (final InjectionException ex) {
+        LOG.log(Level.SEVERE, "Unable to configure driver client.");
+        driverServiceClient.onInitializationException(ex.getCause() != null ? ex.getCause() : ex);
+      } catch (final Throwable t) {
+        if (t.getCause() != null && t.getCause() instanceof InjectionException) {
+          LOG.log(Level.SEVERE, "Unable to configure driver client.");
+          final InjectionException ex = (InjectionException) t.getCause();
+          driverServiceClient.onInitializationException(ex.getCause() != null ? ex.getCause() : ex);
+        } else {
+          throw fatal("Unable run clock.", t);
+        }
+      }
+    } catch (final InjectionException e) {
+      throw fatal("Unable initialize driver service client.", e);
+    }
+
+    ThreadLogger.logThreads(LOG, Level.FINEST, "Threads running after Clock.close():");
+
+    LOG.log(Level.INFO, "Exiting REEFLauncher.main()");
+
+    System.exit(0); // TODO[REEF-1715]: Should be able to exit cleanly at the end of main()
+  }
+
+  /**
+   * Wrap an exception into RuntimeException with a given message,
+   * and write the same message and exception to the log.
+   * @param msg an error message to log and pass into the RuntimeException.
+   * @param t A Throwable exception to log and wrap.
+   * @return a new Runtime exception wrapping a Throwable.
+   */
+  private static RuntimeException fatal(final String msg, final Throwable t) {
+    LOG.log(Level.SEVERE, msg, t);
+    return new RuntimeException(msg, t);
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ActiveContextBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ActiveContextBridge.java
new file mode 100644
index 0000000..02bc9d2
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ActiveContextBridge.java
@@ -0,0 +1,113 @@
+/*
+ * 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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.DriverServiceClient;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.util.Optional;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Active context bridge.
+ */
+@Private
+public final class ActiveContextBridge implements ActiveContext {
+
+  private static final Logger LOG = Logger.getLogger(ActiveContextBridge.class.getName());
+
+  private final DriverServiceClient driverServiceClient;
+
+  private final String contextId;
+
+  private final Optional<String> parentId;
+
+  private final String evaluatorId;
+
+  private final EvaluatorDescriptor evaluatorDescriptor;
+
+  public ActiveContextBridge(
+      final DriverServiceClient driverServiceClient,
+      final String contextId,
+      final Optional<String> parentId,
+      final String evaluatorId,
+      final EvaluatorDescriptor evaluatorDescriptor) {
+    this.driverServiceClient = driverServiceClient;
+    this.contextId = contextId;
+    this.parentId = parentId;
+    this.evaluatorId = evaluatorId;
+    this.evaluatorDescriptor = evaluatorDescriptor;
+  }
+
+  @Override
+  public void close() {
+    LOG.log(Level.INFO, "closing context {0}", this.contextId);
+    this.driverServiceClient.onContextClose(this.contextId);
+  }
+
+  @Override
+  public void submitTask(final Configuration taskConf) {
+    LOG.log(Level.INFO, "submitting task via context {0}", this.contextId);
+    this.driverServiceClient.onContextSubmitTask(this.contextId, taskConf);
+  }
+
+  @Override
+  public void submitContext(final Configuration contextConfiguration) {
+    LOG.log(Level.INFO, "submitting child context via context {0}", this.contextId);
+    this.driverServiceClient.onContextSubmitContext(this.contextId, contextConfiguration);
+  }
+
+  @Override
+  public void submitContextAndService(
+      final Configuration contextConfiguration,
+      final Configuration serviceConfiguration) {
+    throw new UnsupportedOperationException("Service not supported");
+  }
+
+  @Override
+  public void sendMessage(final byte[] message) {
+    LOG.log(Level.INFO, "sending message to context {0}", this.contextId);
+    this.driverServiceClient.onContextMessage(this.contextId, message);
+  }
+
+  @Override
+  public String getEvaluatorId() {
+    return this.evaluatorId;
+  }
+
+  @Override
+  public Optional<String> getParentId() {
+    return this.parentId;
+  }
+
+  @Override
+  public EvaluatorDescriptor getEvaluatorDescriptor() {
+    return this.evaluatorDescriptor;
+  }
+
+  @Override
+  public String getId() {
+    return this.contextId;
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/AllocatedEvaluatorBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/AllocatedEvaluatorBridge.java
new file mode 100644
index 0000000..4369600
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/AllocatedEvaluatorBridge.java
@@ -0,0 +1,146 @@
+/*
+ * 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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.DriverServiceClient;
+import org.apache.reef.bridge.driver.client.JVMClientProcess;
+import org.apache.reef.driver.evaluator.AllocatedEvaluator;
+import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
+import org.apache.reef.driver.evaluator.EvaluatorProcess;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.util.Optional;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Allocated Evaluator Stub.
+ */
+@Private
+public final class AllocatedEvaluatorBridge implements AllocatedEvaluator {
+
+  private final String evaluatorId;
+
+  private final EvaluatorDescriptor evaluatorDescriptor;
+
+  private final DriverServiceClient driverServiceClient;
+
+  private final List<File> addFileList = new ArrayList<>();
+
+  private final List<File> addLibraryList = new ArrayList<>();
+
+  private JVMClientProcess evaluatorProcess = null;
+
+  public AllocatedEvaluatorBridge(
+      final String evaluatorId,
+      final EvaluatorDescriptor evaluatorDescriptor,
+      final DriverServiceClient driverServiceClient) {
+    this.evaluatorId = evaluatorId;
+    this.evaluatorDescriptor = evaluatorDescriptor;
+    this.driverServiceClient = driverServiceClient;
+  }
+
+  @Override
+  public String getId() {
+    return this.evaluatorId;
+  }
+
+  @Override
+  public void addFile(final File file) {
+    this.addFileList.add(file);
+  }
+
+  @Override
+  public void addLibrary(final File file) {
+    this.addLibraryList.add(file);
+  }
+
+  @Override
+  public EvaluatorDescriptor getEvaluatorDescriptor() {
+    return this.evaluatorDescriptor;
+  }
+
+  @Override
+  public void setProcess(final EvaluatorProcess process) {
+    if (process instanceof JVMClientProcess) {
+      this.evaluatorProcess = (JVMClientProcess) process;
+    } else {
+      throw new IllegalArgumentException(JVMClientProcess.class.getCanonicalName() + " required.");
+    }
+  }
+
+  @Override
+  public void close() {
+    this.driverServiceClient.onEvaluatorClose(getId());
+  }
+
+  @Override
+  public void submitTask(final Configuration taskConfiguration) {
+    this.driverServiceClient.onEvaluatorSubmit(
+        getId(),
+        Optional.<Configuration>empty(),
+        Optional.of(taskConfiguration),
+        Optional.ofNullable(this.evaluatorProcess),
+        this.addFileList,
+        this.addLibraryList);
+  }
+
+  @Override
+  public void submitContext(final Configuration contextConfiguration) {
+
+    this.driverServiceClient.onEvaluatorSubmit(
+        getId(),
+        Optional.of(contextConfiguration),
+        Optional.<Configuration>empty(),
+        Optional.ofNullable(this.evaluatorProcess),
+        this.addFileList,
+        this.addLibraryList);
+  }
+
+  @Override
+  public void submitContextAndService(
+      final Configuration contextConfiguration,
+      final Configuration serviceConfiguration) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void submitContextAndTask(
+      final Configuration contextConfiguration,
+      final Configuration taskConfiguration) {
+
+    this.driverServiceClient.onEvaluatorSubmit(
+        getId(),
+        Optional.of(contextConfiguration),
+        Optional.of(taskConfiguration),
+        Optional.ofNullable(this.evaluatorProcess),
+        this.addFileList,
+        this.addLibraryList);
+  }
+
+  @Override
+  public void submitContextAndServiceAndTask(
+      final Configuration contextConfiguration,
+      final Configuration serviceConfiguration,
+      final Configuration taskConfiguration) {
+    throw new UnsupportedOperationException("Service not supported");
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ClosedContextBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ClosedContextBridge.java
new file mode 100644
index 0000000..4527586
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ClosedContextBridge.java
@@ -0,0 +1,77 @@
+/*
+ * 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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.context.ClosedContext;
+import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
+import org.apache.reef.util.Optional;
+
+/**
+ * Closed context bridge.
+ */
+@Private
+public final class ClosedContextBridge implements ClosedContext {
+
+  private final String contextId;
+
+  private final String evaluatorId;
+
+  private final ActiveContext parentContext;
+
+  private final EvaluatorDescriptor evaluatorDescriptor;
+
+  public ClosedContextBridge(
+      final String contextId,
+      final String evaluatorId,
+      final ActiveContext parentContext,
+      final EvaluatorDescriptor evaluatorDescriptor) {
+    this.contextId = contextId;
+    this.evaluatorId = evaluatorId;
+    this.parentContext = parentContext;
+    this.evaluatorDescriptor = evaluatorDescriptor;
+  }
+
+  @Override
+  public ActiveContext getParentContext() {
+    return this.parentContext;
+  }
+
+  @Override
+  public String getId() {
+    return this.contextId;
+  }
+
+  @Override
+  public String getEvaluatorId() {
+    return this.evaluatorId;
+  }
+
+  @Override
+  public Optional<String> getParentId() {
+    return Optional.of(this.parentContext.getId());
+  }
+
+  @Override
+  public EvaluatorDescriptor getEvaluatorDescriptor() {
+    return this.evaluatorDescriptor;
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/CompletedEvaluatorBridge.java
similarity index 64%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/CompletedEvaluatorBridge.java
index 9967d9b..c233efa 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/CompletedEvaluatorBridge.java
@@ -16,20 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.CompletedEvaluator;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Completed Evaluator bridge.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Private
+public final class CompletedEvaluatorBridge implements CompletedEvaluator {
+
+  private final String id;
+
+  public CompletedEvaluatorBridge(final String id) {
+    this.id = id;
+  }
+
+  @Override
+  public String getId() {
+    return this.id;
+  }
 }
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/CompletedTaskBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/CompletedTaskBridge.java
new file mode 100644
index 0000000..77d2379
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/CompletedTaskBridge.java
@@ -0,0 +1,61 @@
+/*
+ * 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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.task.CompletedTask;
+
+/**
+ * Completed task bridge.
+ */
+@Private
+public final class CompletedTaskBridge implements CompletedTask {
+
+  private final String taskId;
+
+  private final ActiveContext context;
+
+  private final byte[] result;
+
+  public CompletedTaskBridge(
+      final String taskId,
+      final ActiveContext context,
+      final byte[] result) {
+    this.taskId = taskId;
+    this.context = context;
+    this.result = result;
+  }
+
+  @Override
+  public ActiveContext getActiveContext() {
+    return this.context;
+  }
+
+  @Override
+  public String getId() {
+    return this.taskId;
+  }
+
+  @Override
+  public byte[] get() {
+    return this.result;
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ContextMessageBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ContextMessageBridge.java
new file mode 100644
index 0000000..f208735
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/ContextMessageBridge.java
@@ -0,0 +1,69 @@
+/*
+ * 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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.ContextMessage;
+
+/**
+ * Context message bridge.
+ */
+@Private
+public final class ContextMessageBridge implements ContextMessage {
+
+  private final String contextId;
+
+  private final String messageSourceId;
+
+  private final long sequenceNumber;
+
+  private final byte[] message;
+
+  public ContextMessageBridge(
+      final String contextId,
+      final String messageSourceId,
+      final long sequenceNumber,
+      final byte[] message) {
+    this.contextId = contextId;
+    this.messageSourceId = messageSourceId;
+    this.sequenceNumber = sequenceNumber;
+    this.message = message;
+  }
+
+  @Override
+  public byte[] get() {
+    return this.message;
+  }
+
+  @Override
+  public String getId() {
+    return this.contextId;
+  }
+
+  @Override
+  public String getMessageSourceID() {
+    return this.messageSourceId;
+  }
+
+  @Override
+  public long getSequenceNumber() {
+    return this.sequenceNumber;
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/FailedContextBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/FailedContextBridge.java
new file mode 100644
index 0000000..9cf0e23
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/FailedContextBridge.java
@@ -0,0 +1,112 @@
+/*
+ * 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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.context.FailedContext;
+import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
+import org.apache.reef.exception.EvaluatorException;
+import org.apache.reef.util.Optional;
+
+/**
+ * Failed context bridge.
+ */
+@Private
+public final class FailedContextBridge implements FailedContext {
+
+  private final String contextId;
+
+  private final String evaluatorId;
+
+  private final String message;
+
+  private final EvaluatorDescriptor evaluatorDescriptor;
+
+  private final Optional<ActiveContext> parentContext;
+
+  private final Optional<Throwable> reason;
+
+  public FailedContextBridge(
+      final String contextId,
+      final String evaluatorId,
+      final String message,
+      final EvaluatorDescriptor evaluatorDescriptor,
+      final Optional<ActiveContext> parentContext,
+      final Optional<Throwable> reason) {
+    this.contextId = contextId;
+    this.evaluatorId = evaluatorId;
+    this.message = message;
+    this.evaluatorDescriptor = evaluatorDescriptor;
+    this.parentContext = parentContext;
+    this.reason = reason;
+  }
+
+  @Override
+  public Optional<ActiveContext> getParentContext() {
+    return this.parentContext;
+  }
+
+  @Override
+  public String getMessage() {
+    return this.message;
+  }
+
+  @Override
+  public Optional<String> getDescription() {
+    return Optional.of(message);
+  }
+
+  @Override
+  public Optional<Throwable> getReason() {
+    return this.reason;
+  }
+
+  @Override
+  public Optional<byte[]> getData() {
+    return Optional.empty();
+  }
+
+  @Override
+  public Throwable asError() {
+    return new EvaluatorException(this.evaluatorId, this.message);
+  }
+
+  @Override
+  public String getEvaluatorId() {
+    return this.evaluatorId;
+  }
+
+  @Override
+  public Optional<String> getParentId() {
+    return this.parentContext.isPresent() ?
+        Optional.of(this.parentContext.get().getId()) : Optional.<String>empty();
+  }
+
+  @Override
+  public EvaluatorDescriptor getEvaluatorDescriptor() {
+    return this.evaluatorDescriptor;
+  }
+
+  @Override
+  public String getId() {
+    return this.contextId;
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/FailedEvaluatorBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/FailedEvaluatorBridge.java
new file mode 100644
index 0000000..64b268e
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/FailedEvaluatorBridge.java
@@ -0,0 +1,75 @@
+/*
+ * 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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.FailedContext;
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.exception.EvaluatorException;
+import org.apache.reef.util.Optional;
+
+import java.util.List;
+
+/**
+ * Failed Evaluator bridge.
+ */
+@Private
+public final class FailedEvaluatorBridge implements FailedEvaluator {
+
+  private final String id;
+
+  private final EvaluatorException evaluatorException;
+
+  private final List<FailedContext> failedContextList;
+
+  private Optional<FailedTask> failedTask;
+
+  public FailedEvaluatorBridge(
+      final String id,
+      final EvaluatorException evaluatorException,
+      final List<FailedContext> failedContextList,
+      final Optional<FailedTask> failedTask) {
+    this.id = id;
+    this.evaluatorException = evaluatorException;
+    this.failedContextList = failedContextList;
+    this.failedTask = failedTask;
+  }
+
+  @Override
+  public EvaluatorException getEvaluatorException() {
+    return this.evaluatorException;
+  }
+
+  @Override
+  public List<FailedContext> getFailedContextList() {
+    return this.failedContextList;
+  }
+
+  @Override
+  public Optional<FailedTask> getFailedTask() {
+    return this.failedTask;
+  }
+
+  @Override
+  public String getId() {
+    return this.id;
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/RunningTaskBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/RunningTaskBridge.java
new file mode 100644
index 0000000..8635930
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/RunningTaskBridge.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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.DriverServiceClient;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.runtime.common.driver.task.TaskRepresenter;
+import org.apache.reef.util.Optional;
+
+/**
+ * Running task bridge.
+ */
+@Private
+public final class RunningTaskBridge implements RunningTask {
+
+  private final DriverServiceClient driverServiceClient;
+
+  private final String taskId;
+
+  private final ActiveContext context;
+
+  public RunningTaskBridge(
+      final DriverServiceClient driverServiceClient,
+      final String taskId,
+      final ActiveContext context) {
+    this.driverServiceClient = driverServiceClient;
+    this.taskId = taskId;
+    this.context = context;
+  }
+
+  @Override
+  public ActiveContext getActiveContext() {
+    return this.context;
+  }
+
+  @Override
+  public void send(final byte[] message) {
+    this.driverServiceClient.onTaskMessage(this.taskId, message);
+  }
+
+  @Override
+  public void suspend(final byte[] message) {
+    this.driverServiceClient.onSuspendTask(this.taskId, Optional.of(message));
+  }
+
+  @Override
+  public void suspend() {
+    this.driverServiceClient.onSuspendTask(this.taskId, Optional.<byte[]>empty());
+  }
+
+  @Override
+  public void close(final byte[] message) {
+    this.driverServiceClient.onTaskClose(this.taskId, Optional.of(message));
+  }
+
+  @Override
+  public void close() {
+    this.driverServiceClient.onTaskClose(this.taskId, Optional.<byte[]>empty());
+  }
+
+  @Override
+  public TaskRepresenter getTaskRepresenter() {
+    throw new UnsupportedOperationException("Not a public API");
+  }
+
+  @Override
+  public String getId() {
+    return this.taskId;
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/SuspendedTaskBridge.java
similarity index 50%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/SuspendedTaskBridge.java
index 9967d9b..05f68dc 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/SuspendedTaskBridge.java
@@ -16,20 +16,43 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.task.SuspendedTask;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Suspended task bridge.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Private
+public final class SuspendedTaskBridge implements SuspendedTask {
+
+  private final String taskId;
+
+  private final ActiveContext context;
+
+  private final byte[] result;
+
+  public SuspendedTaskBridge(final String taskId, final ActiveContext context, final byte[] result) {
+    this.taskId = taskId;
+    this.context = context;
+    this.result = result;
+  }
+
+  @Override
+  public ActiveContext getActiveContext() {
+    return this.context;
+  }
+
+  @Override
+  public byte[] get() {
+    return this.result;
+  }
+
+  @Override
+  public String getId() {
+    return this.taskId;
+  }
 }
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/TaskMessageBridge.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/TaskMessageBridge.java
new file mode 100644
index 0000000..41132a3
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/TaskMessageBridge.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.bridge.driver.client.events;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.task.TaskMessage;
+
+/**
+ * Task message bridge.
+ */
+@Private
+public final class TaskMessageBridge implements TaskMessage {
+
+  private final String taskId;
+
+  private final String contextId;
+
+  private final String messageSourceId;
+
+  private final long sequenceNumber;
+
+  private final byte[] message;
+
+  public TaskMessageBridge(
+      final String taskId,
+      final String contextId,
+      final String messageSourceId,
+      final long sequenceNumber,
+      final byte[] message) {
+    this.taskId = taskId;
+    this.contextId = contextId;
+    this.messageSourceId = messageSourceId;
+    this.sequenceNumber = sequenceNumber;
+    this.message = message;
+  }
+
+  @Override
+  public byte[] get() {
+    return this.message;
+  }
+
+  @Override
+  public String getId() {
+    return this.taskId;
+  }
+
+  @Override
+  public long getSequenceNumber() {
+    return this.sequenceNumber;
+  }
+
+  @Override
+  public String getContextId() {
+    return this.contextId;
+  }
+
+  @Override
+  public String getMessageSourceID() {
+    return this.messageSourceId;
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/package-info.java
index 9967d9b..0ba5ba4 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/events/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * REEF event stubs.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.client.events;
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientGrpcConfiguration.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientGrpcConfiguration.java
new file mode 100644
index 0000000..aaedd5a
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientGrpcConfiguration.java
@@ -0,0 +1,48 @@
+/*
+ * 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.bridge.driver.client.grpc;
+
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.bridge.driver.client.DriverClientService;
+import org.apache.reef.bridge.driver.client.DriverServiceClient;
+import org.apache.reef.bridge.driver.client.grpc.parameters.DriverRegistrationTimeout;
+import org.apache.reef.bridge.driver.client.grpc.parameters.DriverServicePort;
+import org.apache.reef.tang.formats.ConfigurationModule;
+import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
+import org.apache.reef.tang.formats.OptionalParameter;
+import org.apache.reef.tang.formats.RequiredParameter;
+
+/**
+ * Configuration module for Grpc runtime.
+ */
+@Unstable
+public final class DriverClientGrpcConfiguration extends ConfigurationModuleBuilder {
+
+  public static final RequiredParameter<Integer> DRIVER_SERVICE_PORT = new RequiredParameter<>();
+
+  public static final OptionalParameter<Integer> DRIVER_CLIENT_REGISTRATION_TIMEOUT = new OptionalParameter<>();
+
+  public static final ConfigurationModule CONF = new DriverClientGrpcConfiguration()
+      .bindImplementation(DriverClientService.class, GRPCDriverClientService.class)
+      .bindImplementation(DriverServiceClient.class, GRPCDriverServiceClient.class)
+      .bindNamedParameter(DriverServicePort.class, DRIVER_SERVICE_PORT)
+      .bindNamedParameter(DriverRegistrationTimeout.class, DRIVER_CLIENT_REGISTRATION_TIMEOUT)
+      .build();
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/GRPCDriverClientService.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/GRPCDriverClientService.java
new file mode 100644
index 0000000..f68a1cb
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/GRPCDriverClientService.java
@@ -0,0 +1,629 @@
+/*
+ * 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.bridge.driver.client.grpc;
+
+import com.google.common.collect.Lists;
+import io.grpc.Server;
+import io.grpc.ServerBuilder;
+import io.grpc.Status;
+import io.grpc.stub.StreamObserver;
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.DriverClientDispatcher;
+import org.apache.reef.bridge.driver.client.DriverClientService;
+import org.apache.reef.bridge.driver.client.JVMClientProcess;
+import org.apache.reef.bridge.driver.client.events.*;
+import org.apache.reef.bridge.driver.common.grpc.GRPCUtils;
+import org.apache.reef.bridge.driver.common.grpc.ObserverCleanup;
+import org.apache.reef.bridge.proto.*;
+import org.apache.reef.bridge.proto.Void;
+import org.apache.reef.driver.catalog.NodeDescriptor;
+import org.apache.reef.driver.catalog.RackDescriptor;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.context.FailedContext;
+import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
+import org.apache.reef.driver.restart.DriverRestartCompleted;
+import org.apache.reef.driver.restart.DriverRestarted;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.exception.EvaluatorException;
+import org.apache.reef.runtime.common.driver.evaluator.EvaluatorDescriptorBuilderFactory;
+import org.apache.reef.runtime.common.utils.ExceptionCodec;
+import org.apache.reef.tang.InjectionFuture;
+import org.apache.reef.util.Optional;
+import org.apache.reef.wake.remote.ports.TcpPortProvider;
+import org.apache.reef.wake.time.Clock;
+import org.apache.reef.wake.time.Time;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.*;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * The driver client service that accepts incoming messages driver service and
+ * dispatches appropriate objects to the application.
+ */
+@Private
+public final class GRPCDriverClientService extends DriverClientGrpc.DriverClientImplBase
+    implements DriverClientService {
+
+  private static final Logger LOG = Logger.getLogger(GRPCDriverClientService.class.getName());
+
+  private Server server;
+
+  private final Object lock = new Object();
+
+  private final InjectionFuture<Clock> clock;
+
+  private final ExceptionCodec exceptionCodec;
+
+  private final GRPCDriverServiceClient driverServiceClient;
+
+  private final TcpPortProvider tcpPortProvider;
+
+  private final EvaluatorDescriptorBuilderFactory evaluatorDescriptorBuilderFactory;
+
+  private final InjectionFuture<DriverClientDispatcher> clientDriverDispatcher;
+
+  private final Map<String, AllocatedEvaluatorBridge> evaluatorBridgeMap = new HashMap<>();
+
+  private final Map<String, ActiveContextBridge> activeContextBridgeMap = new HashMap<>();
+
+  private int outstandingEvaluatorCount = 0;
+
+  @Inject
+  private GRPCDriverClientService(
+      final EvaluatorDescriptorBuilderFactory evaluatorDescriptorBuilderFactory,
+      final ExceptionCodec exceptionCodec,
+      final GRPCDriverServiceClient driverServiceClient,
+      final TcpPortProvider tcpPortProvider,
+      final InjectionFuture<Clock> clock,
+      final InjectionFuture<DriverClientDispatcher> clientDriverDispatcher) {
+    this.evaluatorDescriptorBuilderFactory = evaluatorDescriptorBuilderFactory;
+    this.exceptionCodec = exceptionCodec;
+    this.driverServiceClient = driverServiceClient;
+    this.tcpPortProvider = tcpPortProvider;
+    this.clock = clock;
+    this.clientDriverDispatcher = clientDriverDispatcher;
+  }
+
+  @Override
+  public void notifyEvaluatorRequest(final int count) {
+    synchronized (this.lock) {
+      this.outstandingEvaluatorCount += count;
+      this.lock.notify();
+    }
+  }
+
+  @Override
+  public void start() throws IOException {
+    for (final int port : this.tcpPortProvider) {
+      try {
+        this.server = ServerBuilder.forPort(port)
+            .addService(this)
+            .build()
+            .start();
+        LOG.log(Level.INFO, "Driver Client Server started, listening on [{0}]", port);
+        break;
+      } catch (final IOException e) {
+        LOG.log(Level.WARNING, "Unable to bind to port [{0}]", port);
+      }
+    }
+    if (this.server == null || this.server.isTerminated()) {
+      throw new IOException("Unable to start gRPC server");
+    }
+    this.driverServiceClient.registerDriverClientService("localhost", this.server.getPort());
+  }
+
+  @Override
+  public void awaitTermination() throws InterruptedException {
+    if (this.server != null) {
+      this.server.awaitTermination();
+    }
+  }
+
+  @Override
+  public void idlenessCheckHandler(final Void request, final StreamObserver<IdleStatus> responseObserver) {
+    if (isIdle()) {
+      LOG.log(Level.INFO, "possibly idle. waiting for some action.");
+      try {
+        synchronized (this.lock) {
+          this.lock.wait(1000); // wait a second
+        }
+      } catch (final InterruptedException e) {
+        LOG.log(Level.FINEST, "Idleness checker wait interrupted");
+      }
+    }
+    responseObserver.onNext(IdleStatus.newBuilder()
+        .setReason("DriverClient checking idleness")
+        .setIsIdle(this.isIdle())
+        .build());
+    responseObserver.onCompleted();
+  }
+
+  @Override
+  public void startHandler(final StartTimeInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "StartHandler at time {0}", request.getStartTime());
+      final StartTime startTime = new StartTime(request.getStartTime());
+      this.clientDriverDispatcher.get().dispatch(startTime);
+    }
+  }
+
+  @Override
+  public void stopHandler(final StopTimeInfo request, final StreamObserver<ExceptionInfo> responseObserver) {
+    try {
+      LOG.log(Level.INFO, "StopHandler at time {0}", request.getStopTime());
+      final StopTime stopTime = new StopTime(request.getStopTime());
+      final Throwable error = this.clientDriverDispatcher.get().dispatch(stopTime);
+      if (error != null) {
+        responseObserver.onNext(GRPCUtils.createExceptionInfo(this.exceptionCodec, error));
+      } else {
+        responseObserver.onNext(ExceptionInfo.newBuilder().setNoError(true).build());
+      }
+    } finally {
+      responseObserver.onCompleted();
+      this.server.shutdown();
+    }
+  }
+
+  @Override
+  public void alarmTrigger(final AlarmTriggerInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Alarm Trigger id {0}", request.getAlarmId());
+      this.clientDriverDispatcher.get().dispatchAlarm(request.getAlarmId());
+    }
+  }
+
+  @Override
+  public void allocatedEvaluatorHandler(final EvaluatorInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      synchronized (this.lock) {
+        assert this.outstandingEvaluatorCount > 0;
+        this.outstandingEvaluatorCount--;
+      }
+      LOG.log(Level.INFO, "Allocated evaluator id {0}", request.getEvaluatorId());
+      final AllocatedEvaluatorBridge eval = new AllocatedEvaluatorBridge(
+          request.getEvaluatorId(),
+          toEvaluatorDescriptor(request.getDescriptorInfo()),
+          this.driverServiceClient);
+      this.evaluatorBridgeMap.put(eval.getId(), eval);
+      this.clientDriverDispatcher.get().dispatch(eval);
+    }
+  }
+
+  @Override
+  public void completedEvaluatorHandler(final EvaluatorInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Completed Evaluator id {0}", request.getEvaluatorId());
+      this.evaluatorBridgeMap.remove(request.getEvaluatorId());
+      this.clientDriverDispatcher.get().dispatch(new CompletedEvaluatorBridge(request.getEvaluatorId()));
+    }
+  }
+
+  @Override
+  public void failedEvaluatorHandler(final EvaluatorInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      if (!this.evaluatorBridgeMap.containsKey(request.getEvaluatorId())) {
+        LOG.log(Level.INFO, "Failed evalautor that we were not allocated");
+        synchronized (this.lock) {
+          if (this.outstandingEvaluatorCount > 0) {
+            this.outstandingEvaluatorCount--;
+          }
+        }
+        return;
+      }
+      LOG.log(Level.INFO, "Failed Evaluator id {0}", request.getEvaluatorId());
+      final AllocatedEvaluatorBridge eval = this.evaluatorBridgeMap.remove(request.getEvaluatorId());
+      final List<FailedContext> failedContextList = new ArrayList<>();
+      if (request.getFailure().getFailedContextsList() != null) {
+        for (final String failedContextId : request.getFailure().getFailedContextsList()) {
+          final ActiveContextBridge context = this.activeContextBridgeMap.get(failedContextId);
+          failedContextList.add(new FailedContextBridge(
+              context.getId(),
+              eval.getId(),
+              request.getFailure().getMessage(),
+              eval.getEvaluatorDescriptor(),
+              Optional.<ActiveContext>ofNullable(this.activeContextBridgeMap.get(context.getParentId().get())),
+              Optional.<Throwable>empty()));
+        }
+        for (final String failedContextId : request.getFailure().getFailedContextsList()) {
+          this.activeContextBridgeMap.remove(failedContextId);
+        }
+      }
+      this.clientDriverDispatcher.get().dispatch(
+          new FailedEvaluatorBridge(
+              eval.getId(),
+              new EvaluatorException(request.getEvaluatorId(), request.getFailure().getMessage()),
+              failedContextList,
+              request.getFailure().getFailedTaskId() != null ?
+                  Optional.of(new FailedTask(
+                      request.getFailure().getFailedTaskId(),
+                      request.getFailure().getMessage(),
+                      Optional.<String>empty(),
+                      Optional.<Throwable>empty(),
+                      Optional.<byte[]>empty(),
+                      Optional.<ActiveContext>empty())) :
+                  Optional.<FailedTask>empty()));
+    }
+  }
+
+  @Override
+  public void activeContextHandler(final ContextInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Active context id {0}", request.getContextId());
+      final AllocatedEvaluatorBridge eval = this.evaluatorBridgeMap.get(request.getEvaluatorId());
+      final ActiveContextBridge context = new ActiveContextBridge(
+          this.driverServiceClient,
+          request.getContextId(),
+          Optional.ofNullable(request.getParentId()),
+          eval.getId(),
+          eval.getEvaluatorDescriptor());
+      this.activeContextBridgeMap.put(context.getId(), context);
+      this.clientDriverDispatcher.get().dispatch(context);
+    }
+  }
+
+  @Override
+  public void closedContextHandler(final ContextInfo request, final StreamObserver<Void> responseObserver) {
+    if (this.activeContextBridgeMap.containsKey(request.getContextId())) {
+      LOG.log(Level.INFO, "Closed context id {0}", request.getContextId());
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        final ActiveContextBridge context = this.activeContextBridgeMap.remove(request.getContextId());
+        this.clientDriverDispatcher.get().dispatch(
+            new ClosedContextBridge(
+                context.getId(),
+                context.getEvaluatorId(),
+                this.activeContextBridgeMap.get(request.getParentId()),
+                context.getEvaluatorDescriptor()));
+      }
+    } else {
+      responseObserver.onError(Status.INTERNAL
+          .withDescription("Unknown context id " + request.getContextId() + " in close")
+          .asRuntimeException());
+    }
+  }
+
+  @Override
+  public void failedContextHandler(final ContextInfo request, final StreamObserver<Void> responseObserver) {
+    if (this.activeContextBridgeMap.containsKey(request.getContextId())) {
+      LOG.log(Level.INFO, "Failed context id {0}", request.getContextId());
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        final ActiveContextBridge context = this.activeContextBridgeMap.remove(request.getContextId());
+        final Optional<ActiveContext> parent =
+            Optional.<ActiveContext>ofNullable(this.activeContextBridgeMap.get(context.getParentId().get()));
+        final Optional<Throwable> reason =
+            this.exceptionCodec.fromBytes(request.getException().getData().toByteArray());
+        this.clientDriverDispatcher.get().dispatch(
+            new FailedContextBridge(
+                context.getId(),
+                context.getEvaluatorId(),
+                request.getException().getMessage(),
+                context.getEvaluatorDescriptor(),
+                parent,
+                reason));
+      }
+    } else {
+      responseObserver.onError(Status.INTERNAL
+          .withDescription("Unknown context id " + request.getContextId() + " in close")
+          .asRuntimeException());
+      responseObserver.onCompleted();
+    }
+  }
+
+  @Override
+  public void contextMessageHandler(final ContextMessageInfo request, final StreamObserver<Void> responseObserver) {
+    if (this.activeContextBridgeMap.containsKey(request.getContextId())) {
+      LOG.log(Level.INFO, "Message context id {0}", request.getContextId());
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        this.clientDriverDispatcher.get().dispatch(
+            new ContextMessageBridge(
+                request.getContextId(),
+                request.getMessageSourceId(),
+                request.getSequenceNumber(),
+                request.getPayload().toByteArray()));
+      }
+    } else {
+      responseObserver.onError(Status.INTERNAL
+          .withDescription("Unknown context id " + request.getContextId() + " in close")
+          .asRuntimeException());
+      responseObserver.onCompleted();
+    }
+  }
+
+  @Override
+  public void runningTaskHandler(final TaskInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Running task id {0}", request.getTaskId());
+      final ContextInfo contextInfo = request.getContext();
+      final ActiveContextBridge context = addContextIfMissing(contextInfo);
+      this.clientDriverDispatcher.get().dispatch(
+          new RunningTaskBridge(this.driverServiceClient, request.getTaskId(), context));
+    }
+  }
+
+  @Override
+  public void failedTaskHandler(final TaskInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Failed task id {0}", request.getTaskId());
+      ActiveContextBridge context = request.hasContext() ?
+          addContextIfMissing(request.getContext()) : null;
+      this.clientDriverDispatcher.get().dispatch(
+          new FailedTask(
+              request.getTaskId(),
+              request.getException().getMessage(),
+              Optional.of(request.getException().getName()),
+              request.getException().getData().isEmpty() ?
+                  Optional.<Throwable>of(new EvaluatorException(request.getException().getMessage())) :
+                  this.exceptionCodec.fromBytes(request.getException().getData().toByteArray()),
+              Optional.<byte[]>empty(),
+              Optional.<ActiveContext>ofNullable(context)));
+    }
+  }
+
+  @Override
+  public void completedTaskHandler(final TaskInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Completed task id {0}", request.getTaskId());
+      final ContextInfo contextInfo = request.getContext();
+      ActiveContextBridge context = addContextIfMissing(contextInfo);
+      this.clientDriverDispatcher.get().dispatch(
+          new CompletedTaskBridge(
+              request.getTaskId(),
+              context,
+              GRPCUtils.toByteArray(request.getResult())));
+    }
+  }
+
+  @Override
+  public void suspendedTaskHandler(final TaskInfo request, final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Suspended task id {0}", request.getTaskId());
+      final ContextInfo contextInfo = request.getContext();
+      ActiveContextBridge context = addContextIfMissing(contextInfo);
+      this.clientDriverDispatcher.get().dispatch(
+          new SuspendedTaskBridge(
+              request.getTaskId(),
+              context,
+              GRPCUtils.toByteArray(request.getResult())));
+    }
+  }
+
+  @Override
+  public void taskMessageHandler(final TaskMessageInfo request, final StreamObserver<Void> responseObserver) {
+    if (this.activeContextBridgeMap.containsKey(request.getContextId())) {
+      LOG.log(Level.INFO, "Message task id {0}", request.getTaskId());
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        this.clientDriverDispatcher.get().dispatch(
+            new TaskMessageBridge(
+                request.getTaskId(),
+                request.getContextId(),
+                request.getMessageSourceId(),
+                request.getSequenceNumber(),
+                request.getPayload().toByteArray()));
+      }
+    } else {
+      responseObserver.onError(Status.INTERNAL
+          .withDescription("Unknown context id: " + request.getContextId())
+          .asRuntimeException());
+      responseObserver.onCompleted();
+    }
+  }
+
+  @Override
+  public void clientMessageHandler(final ClientMessageInfo request, final StreamObserver<Void> responseObserver) {
+    LOG.log(Level.INFO, "Client message");
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      this.clientDriverDispatcher.get().clientMessageDispatch(request.getPayload().toByteArray());
+    }
+  }
+
+  @Override
+  public void clientCloseHandler(final Void request, final StreamObserver<Void> responseObserver) {
+    LOG.log(Level.INFO, "Client close");
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      this.clientDriverDispatcher.get().clientCloseDispatch();
+    }
+  }
+
+  @Override
+  public void clientCloseWithMessageHandler(
+      final ClientMessageInfo request,
+      final StreamObserver<Void> responseObserver) {
+    LOG.log(Level.INFO, "Client close with message");
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      this.clientDriverDispatcher.get().clientCloseWithMessageDispatch(request.getPayload().toByteArray());
+    }
+  }
+
+  @Override
+  public void driverRestartHandler(final DriverRestartInfo request, final StreamObserver<Void> responseObserver) {
+    LOG.log(Level.INFO, "Driver restarted");
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      final DriverRestarted driverRestarted = new DriverRestarted() {
+        @Override
+        public int getResubmissionAttempts() {
+          return request.getResubmissionAttempts();
+        }
+
+        @Override
+        public StartTime getStartTime() {
+          return new StartTime(request.getStartTime().getStartTime());
+        }
+
+        @Override
+        public Set<String> getExpectedEvaluatorIds() {
+          return new HashSet<>(request.getExpectedEvaluatorIdsList());
+        }
+      };
+      this.clientDriverDispatcher.get().dispatchRestart(driverRestarted);
+    }
+  }
+
+  @Override
+  public void driverRestartActiveContextHandler(
+      final ContextInfo request,
+      final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Driver restarted active context {0}", request.getContextId());
+      if (!this.evaluatorBridgeMap.containsKey(request.getEvaluatorId())) {
+        final AllocatedEvaluatorBridge eval = new AllocatedEvaluatorBridge(
+            request.getEvaluatorId(),
+            toEvaluatorDescriptor(request.getEvaluatorDescriptorInfo()),
+            this.driverServiceClient);
+        this.evaluatorBridgeMap.put(eval.getId(), eval);
+      }
+      final ActiveContextBridge context = addContextIfMissing(request);
+      this.clientDriverDispatcher.get().dispatchRestart(context);
+    }
+  }
+
+  @Override
+  public void driverRestartRunningTaskHandler(
+      final TaskInfo request,
+      final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      LOG.log(Level.INFO, "Driver restarted running task {0}", request.getTaskId());
+      if (!this.evaluatorBridgeMap.containsKey(request.getContext().getEvaluatorId())) {
+        final AllocatedEvaluatorBridge eval = new AllocatedEvaluatorBridge(
+            request.getContext().getEvaluatorId(),
+            toEvaluatorDescriptor(request.getContext().getEvaluatorDescriptorInfo()),
+            this.driverServiceClient);
+        this.evaluatorBridgeMap.put(eval.getId(), eval);
+      }
+      final ActiveContextBridge context = addContextIfMissing(request.getContext());
+      this.clientDriverDispatcher.get().dispatchRestart(
+          new RunningTaskBridge(this.driverServiceClient, request.getTaskId(), context));
+    }
+  }
+
+  @Override
+  public void driverRestartCompletedHandler(
+      final DriverRestartCompletedInfo request,
+      final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      this.clientDriverDispatcher.get().dispatchRestart(new DriverRestartCompleted() {
+        @Override
+        public Time getCompletedTime() {
+          return new StopTime(request.getCompletionTime().getStopTime());
+        }
+
+        @Override
+        public boolean isTimedOut() {
+          return request.getIsTimedOut();
+        }
+      });
+    }
+  }
+
+  @Override
+  public void driverRestartFailedEvaluatorHandler(
+      final EvaluatorInfo request,
+      final StreamObserver<Void> responseObserver) {
+    try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+      this.clientDriverDispatcher.get().dispatchRestart(new FailedEvaluatorBridge(
+          request.getEvaluatorId(),
+          new EvaluatorException(request.getFailure() != null ?
+              request.getFailure().getMessage() : "restart failed"),
+          Lists.<FailedContext>newArrayList(),
+          Optional.<FailedTask>empty()));
+    }
+  }
+
+  // Helper methods
+  private boolean isIdle() {
+    LOG.log(Level.INFO, "Clock idle {0}, outstanding evaluators {1}, current evaluators {2}",
+        new Object[] {
+            this.clock.get().isIdle(),
+            this.outstandingEvaluatorCount,
+            this.evaluatorBridgeMap.isEmpty()});
+    return clock.get().isIdle() &&
+        this.outstandingEvaluatorCount == 0 &&
+        this.evaluatorBridgeMap.isEmpty();
+  }
+
+  private ActiveContextBridge addContextIfMissing(final ContextInfo contextInfo) {
+    final String contextId = contextInfo.getContextId();
+    ActiveContextBridge context = this.activeContextBridgeMap.get(contextId);
+    if (context == null) {
+      context = toActiveContext(contextInfo);
+      this.activeContextBridgeMap.put(contextId, context);
+    }
+    return context;
+  }
+
+  private EvaluatorDescriptor toEvaluatorDescriptor(final EvaluatorDescriptorInfo info) {
+    final NodeDescriptor nodeDescriptor = new NodeDescriptor() {
+      @Override
+      public InetSocketAddress getInetSocketAddress() {
+        return InetSocketAddress.createUnresolved(
+            info.getNodeDescriptorInfo().getIpAddress(),
+            info.getNodeDescriptorInfo().getPort());
+      }
+
+      @Override
+      public RackDescriptor getRackDescriptor() {
+        return new RackDescriptor() {
+          @Override
+          public List<NodeDescriptor> getNodes() {
+            return Lists.newArrayList();
+          }
+
+          @Override
+          public String getName() {
+            return info.getNodeDescriptorInfo().getRackName();
+          }
+        };
+      }
+
+      @Override
+      public String getName() {
+        return info.getNodeDescriptorInfo().getHostName();
+      }
+
+      @Override
+      public String getId() {
+        return info.getNodeDescriptorInfo().getId();
+      }
+    };
+    return this.evaluatorDescriptorBuilderFactory.newBuilder()
+        .setNodeDescriptor(nodeDescriptor)
+        .setMemory(info.getMemory())
+        .setNumberOfCores(info.getCores())
+        .setEvaluatorProcess(new JVMClientProcess())
+        .setRuntimeName(info.getRuntimeName())
+        .build();
+  }
+
+  private ActiveContextBridge toActiveContext(final ContextInfo contextInfo) {
+    final AllocatedEvaluatorBridge eval = this.evaluatorBridgeMap.get(contextInfo.getEvaluatorId());
+    return new ActiveContextBridge(
+        this.driverServiceClient,
+        contextInfo.getContextId(),
+        StringUtils.isNotEmpty(contextInfo.getParentId()) ?
+            Optional.of(contextInfo.getParentId()) : Optional.<String>empty(),
+        eval.getId(),
+        eval.getEvaluatorDescriptor());
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/GRPCDriverServiceClient.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/GRPCDriverServiceClient.java
new file mode 100644
index 0000000..70481be
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/GRPCDriverServiceClient.java
@@ -0,0 +1,258 @@
+/*
+ * 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.bridge.driver.client.grpc;
+
+import com.google.protobuf.ByteString;
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.DriverServiceClient;
+import org.apache.reef.bridge.driver.client.JVMClientProcess;
+import org.apache.reef.bridge.driver.client.grpc.parameters.DriverRegistrationTimeout;
+import org.apache.reef.bridge.driver.client.grpc.parameters.DriverServicePort;
+import org.apache.reef.bridge.driver.common.grpc.GRPCUtils;
+import org.apache.reef.bridge.proto.*;
+import org.apache.reef.bridge.proto.Void;
+import org.apache.reef.driver.context.ContextConfiguration;
+import org.apache.reef.driver.evaluator.EvaluatorRequest;
+import org.apache.reef.runtime.common.utils.ExceptionCodec;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.util.Optional;
+
+import javax.inject.Inject;
+import java.io.File;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * The client that exposes methods for communicating back to the
+ * driver service.
+ */
+@Private
+public final class GRPCDriverServiceClient implements DriverServiceClient {
+
+  private static final Logger LOG = Logger.getLogger(GRPCDriverServiceClient.class.getName());
+
+  private final ExceptionCodec exceptionCodec;
+
+  private final ConfigurationSerializer configurationSerializer;
+
+  private final DriverServiceGrpc.DriverServiceFutureStub serviceStub;
+
+  private final int driverRegistrationTimeout;
+
+  @Inject
+  private GRPCDriverServiceClient(
+      final ConfigurationSerializer configurationSerializer,
+      final ExceptionCodec exceptionCodec,
+      @Parameter(DriverServicePort.class) final int driverServicePort,
+      @Parameter(DriverRegistrationTimeout.class) final int driverRegistrationTimeout) {
+    this.driverRegistrationTimeout = driverRegistrationTimeout;
+    this.configurationSerializer = configurationSerializer;
+    this.exceptionCodec = exceptionCodec;
+    final ManagedChannel channel = ManagedChannelBuilder
+        .forAddress("localhost", driverServicePort)
+        .usePlaintext()
+        .build();
+    this.serviceStub = DriverServiceGrpc.newFutureStub(channel);
+  }
+
+  public void registerDriverClientService(final String host, final int port) {
+    LOG.log(Level.INFO, "Driver client register with driver service on port {0}", port);
+    this.serviceStub.registerDriverClient(
+        DriverClientRegistration.newBuilder()
+            .setHost(host)
+            .setPort(port)
+            .build());
+  }
+
+  @Override
+  public void onInitializationException(final Throwable ex) {
+    final Future<Void> callComplete = this.serviceStub.registerDriverClient(
+        DriverClientRegistration.newBuilder()
+            .setException(GRPCUtils.createExceptionInfo(this.exceptionCodec, ex))
+            .build());
+    try {
+      callComplete.get(this.driverRegistrationTimeout, TimeUnit.SECONDS);
+    } catch (final ExecutionException | TimeoutException | InterruptedException e) {
+      throw new RuntimeException("Cannot register driver client", e);
+    }
+  }
+
+  @Override
+  public void onShutdown() {
+    this.serviceStub.shutdown(ShutdownRequest.newBuilder().build());
+  }
+
+  @Override
+  public void onShutdown(final Throwable ex) {
+    this.serviceStub.shutdown(ShutdownRequest.newBuilder()
+        .setException(GRPCUtils.createExceptionInfo(this.exceptionCodec, ex))
+        .build());
+  }
+
+  @Override
+  public void onSetAlarm(final String alarmId, final int timeoutMS) {
+    this.serviceStub.setAlarm(
+        AlarmRequest.newBuilder()
+            .setAlarmId(alarmId)
+            .setTimeoutMs(timeoutMS)
+            .build());
+  }
+
+  @Override
+  public void onEvaluatorRequest(final EvaluatorRequest evaluatorRequest) {
+    this.serviceStub.requestResources(
+        ResourceRequest.newBuilder()
+            .setCores(evaluatorRequest.getNumberOfCores())
+            .setMemorySize(evaluatorRequest.getMegaBytes())
+            .setRelaxLocality(evaluatorRequest.getRelaxLocality())
+            .setResourceCount(evaluatorRequest.getNumber())
+            .setRuntimeName(evaluatorRequest.getRuntimeName())
+            .addAllRackNameList(evaluatorRequest.getRackNames())
+            .addAllNodeNameList(evaluatorRequest.getNodeNames())
+            .build());
+  }
+
+  @Override
+  public void onEvaluatorClose(final String evalautorId) {
+    this.serviceStub.allocatedEvaluatorOp(
+        AllocatedEvaluatorRequest.newBuilder()
+            .setEvaluatorId(evalautorId)
+            .setCloseEvaluator(true)
+            .build());
+  }
+
+  @Override
+  public void onEvaluatorSubmit(
+      final String evaluatorId,
+      final Optional<Configuration> contextConfiguration,
+      final Optional<Configuration> taskConfiguration,
+      final Optional<JVMClientProcess> evaluatorProcess,
+      final List<File> addFileList,
+      final List<File> addLibraryList) {
+    final AllocatedEvaluatorRequest.Builder builder =
+        AllocatedEvaluatorRequest.newBuilder().setEvaluatorId(evaluatorId);
+    for (final File file : addFileList) {
+      builder.addAddFiles(file.getAbsolutePath());
+    }
+    for (final File file : addLibraryList) {
+      builder.addAddLibraries(file.getAbsolutePath());
+    }
+    if (evaluatorProcess.isPresent()) {
+      final JVMClientProcess rawEP = evaluatorProcess.get();
+      builder.setSetProcess(
+          AllocatedEvaluatorRequest.EvaluatorProcessRequest.newBuilder()
+              .setConfigurationFileName(rawEP.getConfigurationFileName())
+              .setMemoryMb(rawEP.getMemory())
+              .setStandardOut(rawEP.getStandardOut())
+              .setStandardErr(rawEP.getStandardErr())
+              .addAllOptions(rawEP.getOptions())
+              .build());
+    }
+    if (contextConfiguration.isPresent()) {
+      builder.setContextConfiguration(
+          this.configurationSerializer.toString(contextConfiguration.get()));
+    } else {
+      builder.setContextConfiguration(this.configurationSerializer.toString(ContextConfiguration.CONF
+          .set(ContextConfiguration.IDENTIFIER, "context-" + evaluatorId)
+          .build()));
+    }
+    if (taskConfiguration.isPresent()) {
+      builder.setTaskConfiguration(
+          this.configurationSerializer.toString(taskConfiguration.get()));
+    }
+    this.serviceStub.allocatedEvaluatorOp(builder.build());
+  }
+
+  // Context Operations
+
+  @Override
+  public void onContextClose(final String contextId) {
+    this.serviceStub.activeContextOp(
+        ActiveContextRequest.newBuilder()
+            .setContextId(contextId)
+            .setCloseContext(true)
+            .build());
+  }
+
+  @Override
+  public void onContextSubmitContext(
+      final String contextId,
+      final Configuration contextConfiguration) {
+    this.serviceStub.activeContextOp(
+        ActiveContextRequest.newBuilder()
+            .setContextId(contextId)
+            .setNewContextRequest(this.configurationSerializer.toString(contextConfiguration))
+            .build());
+  }
+
+  @Override
+  public void onContextSubmitTask(
+      final String contextId,
+      final Configuration taskConfiguration) {
+    this.serviceStub.activeContextOp(
+        ActiveContextRequest.newBuilder()
+            .setContextId(contextId)
+            .setNewTaskRequest(this.configurationSerializer.toString(taskConfiguration))
+            .build());
+  }
+
+  @Override
+  public void onContextMessage(final String contextId, final byte[] message) {
+    this.serviceStub.activeContextOp(
+        ActiveContextRequest.newBuilder()
+            .setContextId(contextId)
+            .setMessage(ByteString.copyFrom(message))
+            .build());
+  }
+
+  // Task operations
+
+  @Override
+  public void onTaskClose(final String taskId, final Optional<byte[]> message) {
+    this.taskOp(RunningTaskRequest.Operation.CLOSE, taskId, message.orElse(null));
+  }
+
+  @Override
+  public void onTaskMessage(final String taskId, final byte[] message) {
+    this.taskOp(RunningTaskRequest.Operation.SEND_MESSAGE, taskId, message);
+  }
+
+  @Override
+  public void onSuspendTask(final String taskId, final Optional<byte[]> message) {
+    this.taskOp(RunningTaskRequest.Operation.SUSPEND, taskId, message.orElse(null));
+  }
+
+  private void taskOp(final RunningTaskRequest.Operation op, final String taskId, final byte[] message) {
+    final RunningTaskRequest.Builder request = RunningTaskRequest.newBuilder().setTaskId(taskId).setOperation(op);
+    if (message != null && message.length > 0) {
+      request.setMessage(ByteString.copyFrom(message));
+    }
+    this.serviceStub.runningTaskOp(request.build());
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/package-info.java
index 9967d9b..4e74386 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * gRPC specific implementations of the driver client bridge.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.client.grpc;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/DriverRegistrationTimeout.java
similarity index 64%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/DriverRegistrationTimeout.java
index 9967d9b..1f690e6 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/DriverRegistrationTimeout.java
@@ -16,20 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client.grpc.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Driver registration timeout i.e., how long to wait for the driver client to successfully register itself.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@NamedParameter(doc = "timeout to use in seconds for driver to successfully register", default_value = "5")
+public final class DriverRegistrationTimeout implements Name<Integer> {
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/DriverServicePort.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/DriverServicePort.java
index 9967d9b..857d94c 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/DriverServicePort.java
@@ -16,20 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
+package org.apache.reef.bridge.driver.client.grpc.parameters;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * gRPC driver service port.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@NamedParameter(doc = "Driver Service Grpc port", short_name = "driver-service-port")
+public final class DriverServicePort implements Name<Integer> {
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/package-info.java
index 9967d9b..d3917ea 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/parameters/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * gRPC specific parameters.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.client.grpc.parameters;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java
index 9967d9b..b94b8e5 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Example (Java) driver client.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.client;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/ClientDriverStopHandler.java
similarity index 58%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/ClientDriverStopHandler.java
index 9967d9b..b19602a 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/ClientDriverStopHandler.java
@@ -16,20 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client.parameters;
+
+import org.apache.reef.bridge.driver.client.DefaultDriverClientStopHandler;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.event.StopTime;
+
+import java.util.Set;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Client driver stop handler.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@NamedParameter(doc ="Java driver client stop handler",
+    default_class = DefaultDriverClientStopHandler.class)
+public final class ClientDriverStopHandler implements Name<Set<EventHandler<StopTime>>> {
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/DriverClientDispatchThreadCount.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/DriverClientDispatchThreadCount.java
index 9967d9b..3a9bf41 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/DriverClientDispatchThreadCount.java
@@ -16,20 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.client.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Driver client dispatcher thread count.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@NamedParameter(doc = "Number of dispatch threads", default_value = "1")
+public class DriverClientDispatchThreadCount implements Name<Integer> {
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/package-info.java
index 9967d9b..4771cf5 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/parameters/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Java bridge driver client specific parameters.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.client.parameters;
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/GRPCUtils.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/GRPCUtils.java
new file mode 100644
index 0000000..75c1887
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/GRPCUtils.java
@@ -0,0 +1,127 @@
+/*
+ * 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.bridge.driver.common.grpc;
+
+import com.google.protobuf.ByteString;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.proto.ContextInfo;
+import org.apache.reef.bridge.proto.EvaluatorDescriptorInfo;
+import org.apache.reef.bridge.proto.ExceptionInfo;
+import org.apache.reef.driver.context.ContextBase;
+import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
+import org.apache.reef.runtime.common.utils.ExceptionCodec;
+import org.apache.reef.util.Optional;
+
+/**
+ * Utility methods for gRPC.
+ */
+@Private
+public final class GRPCUtils {
+
+  private GRPCUtils() {
+  }
+
+  /**
+   * Converts ByteString to byte array.
+   * @param bs ByteString
+   * @return byte array or null if not present
+   */
+  public static byte[] toByteArray(final ByteString bs) {
+    return bs == null || bs.isEmpty() ? null : bs.toByteArray();
+  }
+
+  /**
+   * Converts ByteString into an optional byte array.
+   * @param bs ByteString object
+   * @return Optional of byte array
+   */
+  public static Optional<byte[]> toByteArrayOptional(final ByteString bs) {
+    return Optional.ofNullable(toByteArray(bs));
+  }
+
+  /**
+   * Create exception info from exception object.
+   * @param exceptionCodec to encode exception into bytes
+   * @param ex exception object
+   * @return ExceptionInfo
+   */
+  public static ExceptionInfo createExceptionInfo(final ExceptionCodec exceptionCodec, final Throwable ex)  {
+    return ExceptionInfo.newBuilder()
+        .setName(ex.getCause() != null ? ex.getCause().toString() : ex.toString())
+        .setMessage(ex.getMessage() == null ? ex.toString() : ex.getMessage())
+        .setData(ByteString.copyFrom(exceptionCodec.toBytes(ex)))
+        .build();
+  }
+
+  /**
+   * Create an evaluator descriptor info from an EvalautorDescriptor object.
+   * @param descriptor object
+   * @return EvaluatorDescriptorInfo
+   */
+  public static EvaluatorDescriptorInfo toEvaluatorDescriptorInfo(
+      final EvaluatorDescriptor descriptor) {
+    if (descriptor == null) {
+      return null;
+    }
+    EvaluatorDescriptorInfo.NodeDescriptorInfo nodeDescriptorInfo = descriptor.getNodeDescriptor() == null ? null :
+        EvaluatorDescriptorInfo.NodeDescriptorInfo.newBuilder()
+            .setHostName(descriptor.getNodeDescriptor().getName())
+            .setId(descriptor.getNodeDescriptor().getId())
+            .setIpAddress(descriptor.getNodeDescriptor().getInetSocketAddress().getAddress().toString())
+            .setPort(descriptor.getNodeDescriptor().getInetSocketAddress().getPort())
+            .setRackName(descriptor.getNodeDescriptor().getRackDescriptor() == null ?
+                "" : descriptor.getNodeDescriptor().getRackDescriptor().getName())
+            .build();
+    return EvaluatorDescriptorInfo.newBuilder()
+        .setCores(descriptor.getNumberOfCores())
+        .setMemory(descriptor.getMemory())
+        .setRuntimeName(descriptor.getRuntimeName())
+        .setNodeDescriptorInfo(nodeDescriptorInfo)
+        .build();
+  }
+
+  /**
+   * Create a context info from a context object.
+   * @param context object
+   * @return context info
+   */
+  public static ContextInfo toContextInfo(final ContextBase context) {
+    return toContextInfo(context, null);
+  }
+
+  /**
+   * Create a context info from a context object with an error.
+   * @param context object
+   * @param error info
+   * @return context info
+   */
+  public static ContextInfo toContextInfo(final ContextBase context, final ExceptionInfo error) {
+    final ContextInfo.Builder builder = ContextInfo.newBuilder()
+        .setContextId(context.getId())
+        .setEvaluatorId(context.getEvaluatorId())
+        .setParentId(context.getParentId().orElse(""))
+        .setEvaluatorDescriptorInfo(toEvaluatorDescriptorInfo(
+            context.getEvaluatorDescriptor()));
+    if (error != null) {
+      builder.setException(error);
+    }
+    return builder.build();
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/ObserverCleanup.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/ObserverCleanup.java
new file mode 100644
index 0000000..8a228b1
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/ObserverCleanup.java
@@ -0,0 +1,55 @@
+/*
+ * 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.bridge.driver.common.grpc;
+
+import io.grpc.stub.StreamObserver;
+import org.apache.reef.annotations.audience.Private;
+
+/**
+ * Used to assist with finalizing gRPC server calls.
+ * @param <T> server RPC return value
+ */
+@Private
+public final class ObserverCleanup<T> implements AutoCloseable {
+
+  private final StreamObserver<T> observer;
+  private final T nextValue;
+
+  public static <V> ObserverCleanup<V> of(
+      final StreamObserver<V> observer) {
+    return of(observer, null);
+  }
+
+  public static <V> ObserverCleanup<V> of(
+      final StreamObserver<V> observer, final V nextValue) {
+    return new ObserverCleanup<>(observer, nextValue);
+  }
+
+  private ObserverCleanup(final StreamObserver<T> observer, final T nextValue) {
+    this.observer = observer;
+    this.nextValue = nextValue;
+  }
+
+  @Override
+  public void close() {
+    this.observer.onNext(this.nextValue);
+    this.observer.onCompleted();
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/package-info.java
index 9967d9b..05cceed 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/grpc/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Common gRPC classes.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.common.grpc;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/package-info.java
index 9967d9b..f5ffb63 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/common/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Common driver classes.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.common;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/BridgeDriverLauncher.java
similarity index 55%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/BridgeDriverLauncher.java
index 9967d9b..92de91b 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/BridgeDriverLauncher.java
@@ -16,20 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
+package org.apache.reef.bridge.driver.launch;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+import org.apache.reef.annotations.Unstable;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.tang.exceptions.InjectionException;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * All driver launchers implement this method.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Unstable
+public interface BridgeDriverLauncher {
+
+  /**
+   * Launch the driver with the dynamic {@link ClientProtocol.DriverClientConfiguration}.
+   * @param driverClientConfiguration dynamic driver configuration parameters
+   */
+  LauncherStatus launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration)
+      throws InjectionException;
 }
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.java
new file mode 100644
index 0000000..48593b7
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.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.bridge.driver.launch.azbatch;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.launch.BridgeDriverLauncher;
+import org.apache.reef.bridge.driver.service.DriverServiceConfigurationProvider;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.DriverLauncher;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfiguration;
+import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfigurationCreator;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.exceptions.InjectionException;
+
+import javax.inject.Inject;
+
+/**
+ * This is a bootstrap launcher for Azure Batch for submission from C#. It allows for Java Driver
+ * configuration generation directly on the Driver without need of Java dependency if REST
+ * submission is used.
+ */
+@Private
+public final class AzureBatchLauncher implements BridgeDriverLauncher {
+
+  private final DriverServiceConfigurationProvider driverServiceConfigurationProvider;
+
+  @Inject
+  private AzureBatchLauncher(final DriverServiceConfigurationProvider driverServiceConfigurationProvider) {
+    this.driverServiceConfigurationProvider = driverServiceConfigurationProvider;
+  }
+
+  public LauncherStatus launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration)
+      throws InjectionException {
+    return DriverLauncher.getLauncher(generateConfigurationFromJobSubmissionParameters(driverClientConfiguration))
+        .run(driverServiceConfigurationProvider.getDriverServiceConfiguration(driverClientConfiguration));
+  }
+
+  private static Configuration generateConfigurationFromJobSubmissionParameters(
+      final ClientProtocol.DriverClientConfiguration driverClientConfiguration) {
+    return AzureBatchRuntimeConfigurationCreator.getOrCreateAzureBatchRuntimeConfiguration(
+        driverClientConfiguration.getOperatingSystem() ==
+            ClientProtocol.DriverClientConfiguration.OS.WINDOWS)
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_NAME,
+            driverClientConfiguration.getAzbatchRuntime().getAzureBatchAccountName())
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_KEY,
+            driverClientConfiguration.getAzbatchRuntime().getAzureBatchAccountKey())
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_URI,
+            driverClientConfiguration.getAzbatchRuntime().getAzureBatchAccountUri())
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_POOL_ID,
+            driverClientConfiguration.getAzbatchRuntime().getAzureBatchPoolId())
+        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_NAME,
+            driverClientConfiguration.getAzbatchRuntime().getAzureStorageAccountName())
+        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_KEY,
+            driverClientConfiguration.getAzbatchRuntime().getAzureStorageAccountKey())
+        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_CONTAINER_NAME,
+            driverClientConfiguration.getAzbatchRuntime().getAzureStorageContainerName())
+        .build();
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java
index 9967d9b..e39f432 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Azure Batch driver bridge service launcher.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.launch.azbatch;
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/LocalLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/LocalLauncher.java
new file mode 100644
index 0000000..63ed54f
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/LocalLauncher.java
@@ -0,0 +1,70 @@
+/*
+ * 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.bridge.driver.launch.local;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.launch.BridgeDriverLauncher;
+import org.apache.reef.bridge.driver.service.DriverServiceConfigurationProvider;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.DriverLauncher;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.runtime.local.client.LocalRuntimeConfiguration;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationModule;
+
+import javax.inject.Inject;
+
+/**
+ * Submits a folder containing a Driver to the local runtime.
+ */
+@Private
+public final class LocalLauncher implements BridgeDriverLauncher {
+
+  private final DriverServiceConfigurationProvider driverServiceConfigurationProvider;
+
+  @Inject
+  private LocalLauncher(final DriverServiceConfigurationProvider driverServiceConfigurationProvider) {
+    this.driverServiceConfigurationProvider = driverServiceConfigurationProvider;
+  }
+
+  public LauncherStatus launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration)
+      throws InjectionException {
+    ConfigurationModule localRuntimeCM = LocalRuntimeConfiguration.CONF;
+    if (driverClientConfiguration.getLocalRuntime().getMaxNumberOfEvaluators() > 0) {
+      localRuntimeCM = localRuntimeCM.set(LocalRuntimeConfiguration.MAX_NUMBER_OF_EVALUATORS,
+          driverClientConfiguration.getLocalRuntime().getMaxNumberOfEvaluators());
+    }
+    if (StringUtils.isNotEmpty(driverClientConfiguration.getLocalRuntime().getRuntimeRootFolder())) {
+      localRuntimeCM = localRuntimeCM.set(LocalRuntimeConfiguration.RUNTIME_ROOT_FOLDER,
+          driverClientConfiguration.getLocalRuntime().getRuntimeRootFolder());
+    }
+    if (driverClientConfiguration.getLocalRuntime().getJvmHeapSlack() > 0.0) {
+      localRuntimeCM = localRuntimeCM.set(LocalRuntimeConfiguration.JVM_HEAP_SLACK,
+          driverClientConfiguration.getLocalRuntime().getJvmHeapSlack());
+    }
+    if (StringUtils.isNotEmpty(driverClientConfiguration.getDriverJobSubmissionDirectory())) {
+      localRuntimeCM = localRuntimeCM.set(LocalRuntimeConfiguration.RUNTIME_ROOT_FOLDER,
+          driverClientConfiguration.getDriverJobSubmissionDirectory());
+    }
+    return DriverLauncher
+        .getLauncher(localRuntimeCM.build())
+        .run(driverServiceConfigurationProvider.getDriverServiceConfiguration(driverClientConfiguration));
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java
index 9967d9b..8359938 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Local driver bridge service launcher.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.launch.local;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java
index 9967d9b..2eb3422 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Driver launchers for the bridge service on various runtimes.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.launch;
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/YarnLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/YarnLauncher.java
new file mode 100644
index 0000000..58367ef
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/YarnLauncher.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.bridge.driver.launch.yarn;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.launch.BridgeDriverLauncher;
+import org.apache.reef.bridge.driver.service.DriverServiceConfigurationProvider;
+import org.apache.reef.bridge.driver.service.grpc.GRPCDriverServiceConfigurationProvider;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.DriverLauncher;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.runtime.yarn.client.YarnClientConfiguration;
+import org.apache.reef.runtime.yarn.driver.parameters.FileSystemUrl;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+
+import javax.inject.Inject;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * This is a bootstrap launcher for YARN for submission from the bridge. It allows for Java Driver
+ * configuration generation directly on the Driver without need of Java dependency if REST
+ * submission is used.
+ */
+@Private
+public final class YarnLauncher implements BridgeDriverLauncher {
+  private static final Logger LOG = Logger.getLogger(YarnLauncher.class.getName());
+
+  @Inject
+  private YarnLauncher() {
+  }
+
+  public LauncherStatus launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration)
+      throws InjectionException {
+    final DriverServiceConfigurationProvider driverConfigurationProvider =
+        Tang.Factory.getTang().newInjector(Tang.Factory.getTang().newConfigurationBuilder()
+            .bindImplementation(DriverServiceConfigurationProvider.class,
+                GRPCDriverServiceConfigurationProvider.class)
+            .build()
+        ).getInstance(DriverServiceConfigurationProvider.class);
+    Configuration yarnConfiguration = YarnClientConfiguration.CONF
+        .set(YarnClientConfiguration.UNMANAGED_DRIVER,
+            driverClientConfiguration.getYarnRuntime().getUnmangedDriver())
+        .set(YarnClientConfiguration.YARN_PRIORITY, driverClientConfiguration.getYarnRuntime().getPriority())
+        .set(YarnClientConfiguration.JVM_HEAP_SLACK, 0.0)
+        .build();
+    if (StringUtils.isNotEmpty(driverClientConfiguration.getYarnRuntime().getFilesystemUrl())) {
+      final JavaConfigurationBuilder providerConfig = Tang.Factory.getTang().newConfigurationBuilder()
+          .bindNamedParameter(FileSystemUrl.class, driverClientConfiguration.getYarnRuntime().getFilesystemUrl());
+      yarnConfiguration = Configurations.merge(yarnConfiguration, providerConfig.build());
+    }
+    final LauncherStatus status = DriverLauncher.getLauncher(yarnConfiguration)
+        .run(driverConfigurationProvider.getDriverServiceConfiguration(driverClientConfiguration));
+    LOG.log(Level.INFO, "Job complete status: {0}", status);
+    if (status.getError().isPresent()) {
+      LOG.log(Level.SEVERE, "Job completed with error", status.getError().get());
+    }
+    return status;
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java
index 9967d9b..40a715f 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * YARN driver bridge service launcher.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.launch.yarn;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverClientException.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverClientException.java
index 9967d9b..a657680 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverClientException.java
@@ -16,20 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.service;
+
+import org.apache.reef.annotations.Unstable;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * An exception thrown by the driver client.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Unstable
+public final class DriverClientException extends Exception {
+
+  public DriverClientException(final String message) {
+    super(message);
+  }
 }
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverService.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverService.java
new file mode 100644
index 0000000..b1bce16
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverService.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.bridge.driver.service;
+
+import org.apache.reef.annotations.audience.Private;
+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.FailedEvaluator;
+import org.apache.reef.driver.restart.DriverRestartCompleted;
+import org.apache.reef.driver.restart.DriverRestarted;
+import org.apache.reef.driver.task.*;
+import org.apache.reef.runtime.common.driver.idle.DriverIdlenessSource;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+
+/**
+ * Interface implemented by a Driver Service.
+ */
+@Private
+public interface DriverService extends DriverIdlenessSource {
+
+  /**
+   * Driver restart handler.
+   * @param restart event
+   */
+  void driverRestarted(final DriverRestarted restart);
+
+  /**
+   * Restart running task.
+   * @param task running
+   */
+  void restartRunningTask(final RunningTask task);
+
+  /**
+   * Restart active context.
+   * @param context restart
+   */
+  void restartActiveContext(final ActiveContext context);
+
+  /**
+   * Driver restart completed.
+   * @param restartCompleted event
+   */
+  void driverRestartCompleted(final DriverRestartCompleted restartCompleted);
+
+  /**
+   * Failed to restart evaluator.
+   * @param evaluator that failed.
+   */
+  void restartFailedEvalautor(final FailedEvaluator evaluator);
+
+  /**
+   * Handle start time event.
+   * @param startTime event
+   */
+  void startHandler(final StartTime startTime);
+
+  /**
+   * Handle stop event.
+   * @param stopTime event
+   */
+  void stopHandler(final StopTime stopTime);
+
+  /**
+   * Handle allocated evaluator event.
+   * @param eval allocated
+   */
+  void allocatedEvaluatorHandler(final AllocatedEvaluator eval);
+
+  /**
+   * Handle completed evaluator event.
+   * @param eval that completed
+   */
+  void completedEvaluatorHandler(final CompletedEvaluator eval);
+
+  /**
+   * Handle failed evaluator event.
+   * @param eval that failed
+   */
+  void failedEvaluatorHandler(final FailedEvaluator eval);
+
+  /**
+   * Handle active context.
+   * @param context activated
+   */
+  void activeContextHandler(final ActiveContext context);
+
+  /**
+   * Handle closed context event.
+   * @param context that closed
+   */
+  void closedContextHandler(final ClosedContext context);
+
+  /**
+   * Handle context message event.
+   * @param message sent by context
+   */
+  void contextMessageHandler(final ContextMessage message);
+
+  /**
+   * Handled failed context event.
+   * @param context that failed
+   */
+  void failedContextHandler(final FailedContext context);
+
+  /**
+   * Handle running task event.
+   * @param task that is now running
+   */
+  void runningTaskHandler(final RunningTask task);
+
+  /**
+   * Handle failed task event.
+   * @param task that failed
+   */
+  void failedTaskHandler(final FailedTask task);
+
+  /**
+   * Handle completed task event.
+   * @param task that completed
+   */
+  void completedTaskHandler(final CompletedTask task);
+
+  /**
+   * Handle suspended task event.
+   * @param task that is suspended
+   */
+  void suspendedTaskHandler(final SuspendedTask task);
+
+  /**
+   * Handle task message event.
+   * @param message sent by task
+   */
+  void taskMessageHandler(final TaskMessage message);
+
+  /**
+   * Handle client message event.
+   * @param message sent by client
+   */
+  void clientMessageHandler(final byte[] message);
+
+  /**
+   * Handle client close event.
+   */
+  void clientCloseHandler();
+
+  /**
+   * Handle client close event with message.
+   * @param message sent by client
+   */
+  void clientCloseWithMessageHandler(final byte[] message);
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceConfiguration.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceConfiguration.java
new file mode 100644
index 0000000..be422a3
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceConfiguration.java
@@ -0,0 +1,124 @@
+/*
+ * 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.bridge.driver.service;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.service.parameters.DriverClientCommand;
+import org.apache.reef.client.DriverConfiguration;
+import org.apache.reef.client.DriverRestartConfiguration;
+import org.apache.reef.driver.parameters.DriverIdleSources;
+import org.apache.reef.io.network.naming.NameServerConfiguration;
+import org.apache.reef.runtime.common.driver.client.JobStatusHandler;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.formats.ConfigurationModule;
+import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
+import org.apache.reef.tang.formats.RequiredImpl;
+import org.apache.reef.tang.formats.RequiredParameter;
+import org.apache.reef.webserver.HttpHandlerConfiguration;
+import org.apache.reef.webserver.HttpServerReefEventHandler;
+import org.apache.reef.webserver.ReefEventStateManager;
+
+/**
+ * Binds all driver bridge service handlers to the driver.
+ */
+@Private
+public final class DriverServiceConfiguration extends ConfigurationModuleBuilder {
+
+  public static final RequiredImpl<DriverService> DRIVER_SERVICE_IMPL = new RequiredImpl<>();
+
+  public static final RequiredParameter<String> DRIVER_CLIENT_COMMAND = new RequiredParameter<>();
+
+  /** Configuration module that binds all driver handlers. */
+  public static final ConfigurationModule CONF = new DriverServiceConfiguration()
+      .bindImplementation(DriverService.class, DRIVER_SERVICE_IMPL)
+      .bindNamedParameter(DriverClientCommand.class, DRIVER_CLIENT_COMMAND)
+      .bindSetEntry(DriverIdleSources.class, DriverService.class)
+      .build();
+
+  public static final ConfigurationModule STATIC_DRIVER_CONF_MODULE = DriverConfiguration.CONF
+      .set(DriverConfiguration.ON_DRIVER_STARTED, DriverServiceHandlers.StartHandler.class)
+      .set(DriverConfiguration.ON_DRIVER_STOP, DriverServiceHandlers.StopHandler.class)
+      .set(DriverConfiguration.ON_EVALUATOR_ALLOCATED, DriverServiceHandlers.AllocatedEvaluatorHandler.class)
+      .set(DriverConfiguration.ON_EVALUATOR_COMPLETED, DriverServiceHandlers.CompletedEvaluatorHandler.class)
+      .set(DriverConfiguration.ON_EVALUATOR_FAILED, DriverServiceHandlers.FailedEvaluatorHandler.class)
+      .set(DriverConfiguration.ON_CONTEXT_ACTIVE, DriverServiceHandlers.ActiveContextHandler.class)
+      .set(DriverConfiguration.ON_CONTEXT_CLOSED, DriverServiceHandlers.ClosedContextHandler.class)
+      .set(DriverConfiguration.ON_CONTEXT_FAILED, DriverServiceHandlers.ContextFailedHandler.class)
+      .set(DriverConfiguration.ON_CONTEXT_MESSAGE, DriverServiceHandlers.ContextMessageHandler.class)
+      .set(DriverConfiguration.ON_TASK_RUNNING, DriverServiceHandlers.RunningTaskHandler.class)
+      .set(DriverConfiguration.ON_TASK_COMPLETED, DriverServiceHandlers.CompletedTaskHandler.class)
+      .set(DriverConfiguration.ON_TASK_FAILED, DriverServiceHandlers.FailedTaskHandler.class)
+      .set(DriverConfiguration.ON_TASK_SUSPENDED, DriverServiceHandlers.SuspendedTaskHandler.class)
+      .set(DriverConfiguration.ON_TASK_MESSAGE, DriverServiceHandlers.TaskMessageHandler.class)
+      .set(DriverConfiguration.ON_CLIENT_MESSAGE, DriverServiceHandlers.ClientMessageHandler.class)
+      .set(DriverConfiguration.ON_CLIENT_CLOSED, DriverServiceHandlers.ClientCloseHandler.class)
+      .set(DriverConfiguration.ON_CLIENT_CLOSED_MESSAGE, DriverServiceHandlers.ClientCloseWithMessageHandler.class);
+
+  /**
+   * The HTTP Server configuration assumed by the bridge.
+   */
+  private static final Configuration HTTP_SERVER_CONFIGURATION = Configurations.merge(
+      HttpHandlerConfiguration.CONF
+          .set(HttpHandlerConfiguration.HTTP_HANDLERS, HttpServerReefEventHandler.class)
+          // Add the http status handler.
+          .set(HttpHandlerConfiguration.HTTP_HANDLERS, DriverStatusHTTPHandler.class)
+          .build(),
+      org.apache.reef.client.DriverServiceConfiguration.CONF
+          .set(org.apache.reef.client.DriverServiceConfiguration.ON_EVALUATOR_ALLOCATED,
+              ReefEventStateManager.AllocatedEvaluatorStateHandler.class)
+          .set(org.apache.reef.client.DriverServiceConfiguration.ON_CONTEXT_ACTIVE,
+              ReefEventStateManager.ActiveContextStateHandler.class)
+          .set(org.apache.reef.client.DriverServiceConfiguration.ON_TASK_RUNNING,
+              ReefEventStateManager.TaskRunningStateHandler.class)
+          .set(org.apache.reef.client.DriverServiceConfiguration.ON_DRIVER_STARTED,
+              ReefEventStateManager.StartStateHandler.class)
+          .set(org.apache.reef.client.DriverServiceConfiguration.ON_DRIVER_STOP,
+              ReefEventStateManager.StopStateHandler.class)
+          .build(),
+      DriverRestartConfiguration.CONF
+          .set(DriverRestartConfiguration.ON_DRIVER_RESTARTED,
+              ReefEventStateManager.DriverRestartHandler.class)
+          .set(DriverRestartConfiguration.ON_DRIVER_RESTART_CONTEXT_ACTIVE,
+              ReefEventStateManager.DriverRestartActiveContextStateHandler.class)
+          .set(DriverRestartConfiguration.ON_DRIVER_RESTART_TASK_RUNNING,
+              ReefEventStateManager.DriverRestartTaskRunningStateHandler.class)
+          .build(),
+      // Bind the HTTP handler for job status
+      Tang.Factory.getTang().newConfigurationBuilder()
+          .bindImplementation(JobStatusHandler.class, DriverStatusHTTPHandler.class)
+          .build()
+  );
+
+  /**
+   * The name server configuration assumed by the bridge.
+   */
+  private static final Configuration NAME_SERVER_CONFIGURATION = NameServerConfiguration.CONF
+      .set(NameServerConfiguration.NAME_SERVICE_PORT, 0)
+      .build();
+
+  /**
+   * The driver configuration assumed by the the bridge.
+   */
+  public static final Configuration HTTP_AND_NAMESERVER = Configurations.merge(
+      HTTP_SERVER_CONFIGURATION,
+      NAME_SERVER_CONFIGURATION
+  );
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceConfigurationProvider.java
similarity index 55%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceConfigurationProvider.java
index 9967d9b..88a2038 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceConfigurationProvider.java
@@ -16,20 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.driver.service;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.service.grpc.GRPCDriverServiceConfigurationProvider;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.annotations.DefaultImplementation;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Configuration provider for the driver service.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@Private
+@DefaultImplementation(GRPCDriverServiceConfigurationProvider.class)
+public interface DriverServiceConfigurationProvider {
+
+  Configuration getDriverServiceConfiguration(final ClientProtocol.DriverClientConfiguration driverClientConfiguration);
 }
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceHandlers.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceHandlers.java
new file mode 100644
index 0000000..7e1b19c
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverServiceHandlers.java
@@ -0,0 +1,304 @@
+/*
+ * 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.bridge.driver.service;
+
+import org.apache.reef.annotations.audience.DriverSide;
+import org.apache.reef.annotations.audience.Private;
+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.FailedEvaluator;
+import org.apache.reef.driver.restart.DriverRestartCompleted;
+import org.apache.reef.driver.restart.DriverRestarted;
+import org.apache.reef.driver.task.*;
+import org.apache.reef.tang.annotations.Unit;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+
+import javax.inject.Inject;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Contains Java side event handlers that perform
+ * hand-off with the driver client side.
+ */
+@Unit
+@Private
+@DriverSide
+public final class DriverServiceHandlers {
+
+  private static final Logger LOG = Logger.getLogger(DriverServiceHandlers.class.getName());
+
+  private final DriverService driverBridgeService;
+
+  @Inject
+  private DriverServiceHandlers(
+      final DriverService driverBridgeService) {
+    this.driverBridgeService = driverBridgeService;
+  }
+
+  /**
+   * Job Driver is ready and the clock is set up: request the evaluators.
+   */
+  public final class StartHandler implements EventHandler<StartTime> {
+    @Override
+    public void onNext(final StartTime startTime) {
+      LOG.log(Level.INFO, "JavaBridge: Start Driver");
+      DriverServiceHandlers.this.driverBridgeService.startHandler(startTime);
+    }
+  }
+
+  /**
+   * Job Driver is is shutting down: write to the log.
+   */
+  public final class StopHandler implements EventHandler<StopTime> {
+    @Override
+    public void onNext(final StopTime stopTime) {
+      LOG.log(Level.INFO, "JavaBridge: Stop Driver");
+      DriverServiceHandlers.this.driverBridgeService.stopHandler(stopTime);
+    }
+  }
+
+  /**
+   * Receive notification that an Evaluator had been allocated,
+   * and submitTask a new Task in that Evaluator.
+   */
+  public final class AllocatedEvaluatorHandler implements EventHandler<AllocatedEvaluator> {
+    @Override
+    public void onNext(final AllocatedEvaluator eval) {
+      LOG.log(Level.INFO, "JavaBridge: Allocated Evaluator {0}", eval.getId());
+      DriverServiceHandlers.this.driverBridgeService.allocatedEvaluatorHandler(eval);
+    }
+  }
+
+  /**
+   * Completed evaluator handler.
+   */
+  public final class CompletedEvaluatorHandler implements EventHandler<CompletedEvaluator> {
+    @Override
+    public void onNext(final CompletedEvaluator eval) {
+      LOG.log(Level.INFO, "JavaBridge: Completed Evaluator {0}", eval.getId());
+      DriverServiceHandlers.this.driverBridgeService.completedEvaluatorHandler(eval);
+    }
+  }
+
+  /**
+   * Failed evaluator handler.
+   */
+  public final class FailedEvaluatorHandler implements EventHandler<FailedEvaluator> {
+    @Override
+    public void onNext(final FailedEvaluator eval) {
+      LOG.log(Level.INFO, "JavaBridge: Failed Evaluator {0}", eval.getId());
+      DriverServiceHandlers.this.driverBridgeService.failedEvaluatorHandler(eval);
+    }
+  }
+
+  /**
+   * Receive notification that the Context is active.
+   */
+  public final class ActiveContextHandler implements EventHandler<ActiveContext> {
+    @Override
+    public void onNext(final ActiveContext context) {
+      LOG.log(Level.INFO, "JavaBridge: Active Context {0}", context.getId());
+      DriverServiceHandlers.this.driverBridgeService.activeContextHandler(context);
+    }
+  }
+
+  /**
+   * Received notification that the Context is closed.
+   */
+  public final class ClosedContextHandler implements EventHandler<ClosedContext> {
+    @Override
+    public void onNext(final ClosedContext context) {
+      LOG.log(Level.INFO, "JavaBridge: Closed Context {0}", context.getId());
+      DriverServiceHandlers.this.driverBridgeService.closedContextHandler(context);
+    }
+  }
+
+  /**
+   * Received a message from the context.
+   */
+  public final class ContextMessageHandler implements EventHandler<ContextMessage> {
+    @Override
+    public void onNext(final ContextMessage message) {
+      LOG.log(Level.INFO, "JavaBridge: Context Message id {0}", message.getId());
+      DriverServiceHandlers.this.driverBridgeService.contextMessageHandler(message);
+    }
+  }
+
+  /**
+   * Received notification that the Context failed.
+   */
+  public final class ContextFailedHandler implements EventHandler<FailedContext> {
+    @Override
+    public void onNext(final FailedContext context) {
+      LOG.log(Level.INFO, "JavaBridge: Context Failed {0}", context.getId());
+      DriverServiceHandlers.this.driverBridgeService.failedContextHandler(context);
+    }
+  }
+
+  /**
+   * Receive notification that the Task is running.
+   */
+  public final class RunningTaskHandler implements EventHandler<RunningTask> {
+    @Override
+    public void onNext(final RunningTask task) {
+      LOG.log(Level.INFO, "JavaBridge: Running Task {0}", task.getId());
+      DriverServiceHandlers.this.driverBridgeService.runningTaskHandler(task);
+    }
+  }
+
+  /**
+   * Received notification that the Task failed.
+   */
+  public final class FailedTaskHandler implements EventHandler<FailedTask> {
+    @Override
+    public void onNext(final FailedTask task) {
+      LOG.log(Level.INFO, "JavaBridge: Failed Task {0}", task.getId());
+      DriverServiceHandlers.this.driverBridgeService.failedTaskHandler(task);
+    }
+  }
+
+  /**
+   * Receive notification that the Task has completed successfully.
+   */
+  public final class CompletedTaskHandler implements EventHandler<CompletedTask> {
+    @Override
+    public void onNext(final CompletedTask task) {
+      LOG.log(Level.INFO, "JavaBridge: Completed Task {0}", task.getId());
+      DriverServiceHandlers.this.driverBridgeService.completedTaskHandler(task);
+    }
+  }
+
+  /**
+   * Received notification that the Task was suspended.
+   */
+  public final class SuspendedTaskHandler implements EventHandler<SuspendedTask> {
+    @Override
+    public void onNext(final SuspendedTask task) {
+      LOG.log(Level.INFO, "JavaBridge: Suspended Task {0}", task.getId());
+      DriverServiceHandlers.this.driverBridgeService.suspendedTaskHandler(task);
+    }
+  }
+
+  /**
+   * Received a message from the task.
+   */
+  public final class TaskMessageHandler implements EventHandler<TaskMessage> {
+    @Override
+    public void onNext(final TaskMessage message) {
+      LOG.log(Level.INFO, "JavaBridge: Message from Task {0}", message.getId());
+      DriverServiceHandlers.this.driverBridgeService.taskMessageHandler(message);
+    }
+  }
+
+  /**
+   * Received a message from the client.
+   */
+  public final class ClientMessageHandler implements EventHandler<byte[]> {
+    @Override
+    public void onNext(final byte[] message) {
+      LOG.log(Level.INFO, "JavaBridge: Message from Client");
+      DriverServiceHandlers.this.driverBridgeService.clientMessageHandler(message);
+    }
+  }
+
+  /**
+   * Received a close event from the client.
+   */
+  public final class ClientCloseHandler implements EventHandler<Void> {
+    @Override
+    public void onNext(final Void value) {
+      LOG.log(Level.INFO, "JavaBridge: Close event from Client");
+      DriverServiceHandlers.this.driverBridgeService.clientCloseHandler();
+    }
+  }
+
+  /**
+   * Received a close event with message.
+   */
+  public final class ClientCloseWithMessageHandler implements EventHandler<byte[]> {
+    @Override
+    public void onNext(final byte[] message) {
+      LOG.log(Level.INFO, "JavaBridge: Close event with messages from Client");
+      DriverServiceHandlers.this.driverBridgeService.clientCloseWithMessageHandler(message);
+    }
+  }
+
+  /**
+   * Job driver is restarted after previous crash.
+   */
+  public final class DriverRestartHandler implements EventHandler<DriverRestarted> {
+    @Override
+    public void onNext(final DriverRestarted driverRestarted) {
+      LOG.log(Level.INFO, "JavaBridge: driver restarted");
+      DriverServiceHandlers.this.driverBridgeService.driverRestarted(driverRestarted);
+    }
+  }
+
+  /**
+   * Receive notification that an context is active on Evaluator when the driver restarted.
+   */
+  public final class DriverRestartActiveContextHandler implements EventHandler<ActiveContext> {
+    @Override
+    public void onNext(final ActiveContext context) {
+      LOG.log(Level.INFO, "JavaBridge: driver restart active context {0}", context.getId());
+      DriverServiceHandlers.this.driverBridgeService.restartActiveContext(context);
+    }
+  }
+
+  /**
+   * Receive notification that the Task is running when driver restarted.
+   */
+  public final class DriverRestartRunningTaskHandler implements EventHandler<RunningTask> {
+    @Override
+    public void onNext(final RunningTask task) {
+      LOG.log(Level.INFO, "JavaBridge: driver restart running task {0}", task.getId());
+      DriverServiceHandlers.this.driverBridgeService.restartRunningTask(task);
+    }
+  }
+
+  /**
+   * Receive notification that driver restart has completed.
+   */
+  public final class DriverRestartCompletedHandler implements EventHandler<DriverRestartCompleted> {
+    @Override
+    public void onNext(final DriverRestartCompleted driverRestartCompleted) {
+      LOG.log(Level.INFO, "JavaBridge: driver restart completed at {0}",
+          driverRestartCompleted.getCompletedTime());
+      DriverServiceHandlers.this.driverBridgeService.driverRestartCompleted(driverRestartCompleted);
+    }
+  }
+
+  /**
+   * Receive notification that the entire Evaluator had failed on Driver Restart.
+   */
+  public final class DriverRestartFailedEvaluatorHandler implements EventHandler<FailedEvaluator> {
+    @Override
+    public void onNext(final FailedEvaluator eval) {
+      LOG.log(Level.INFO, "JavaBridge: driver restart failed evaluator {0}", eval.getId());
+      DriverServiceHandlers.this.driverBridgeService.restartFailedEvalautor(eval);
+    }
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverStatusHTTPHandler.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverStatusHTTPHandler.java
new file mode 100644
index 0000000..2336eff
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/DriverStatusHTTPHandler.java
@@ -0,0 +1,200 @@
+/*
+ * 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.bridge.driver.service;
+
+import org.apache.reef.bridge.driver.service.parameters.HTTPStatusAlarmInterval;
+import org.apache.reef.bridge.driver.service.parameters.HTTPStatusNumberOfRetries;
+import org.apache.reef.proto.ReefServiceProtos;
+import org.apache.reef.runtime.common.driver.client.JobStatusHandler;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.Clock;
+import org.apache.reef.wake.time.event.Alarm;
+import org.apache.reef.webserver.HttpHandler;
+import org.apache.reef.webserver.ParsedHttpRequest;
+
+import javax.inject.Inject;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Driver service status http handler.
+ */
+final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
+
+  private static final Logger LOG = Logger.getLogger(DriverStatusHTTPHandler.class.getName());
+
+  /**
+   * The URI under which this handler answers.
+   */
+  private String uriSpecification = "driverstatus";
+
+  /**
+   * A queue of messages to be sent to the client.
+   */
+  private final Queue<ReefServiceProtos.JobStatusProto> statusMessagesToSend = new LinkedList<>();
+
+  /**
+   * The last status received by this object in its role as JobStatusHandler.
+   */
+  private ReefServiceProtos.JobStatusProto lastStatus = null;
+
+  /**
+   * The clock is used to schedule a check whether the handler has been called.
+   */
+  private final Clock clock;
+
+  /**
+   * The maximum number of times the AlarmHandler will be scheduled.
+   */
+  private final int maxNumberOfRetries;
+
+  /**
+   * The interval between alarms.
+   */
+  private final int alarmInterval;
+
+  /**
+   * The current retry.
+   */
+  private int retry = 0;
+
+  /**
+   * The alarm handler to keep the Clock alive until the status has been requested once.
+   */
+  private final EventHandler<Alarm> alarmHandler = new EventHandler<Alarm>() {
+    @Override
+    public void onNext(final Alarm value) {
+      scheduleAlarm();
+    }
+  };
+
+  /**
+   * Whether or not this handler was called at least once via HTTP.
+   */
+  private boolean wasCalledViaHTTP = false;
+
+  @Inject
+  DriverStatusHTTPHandler(final Clock clock,
+                          @Parameter(HTTPStatusNumberOfRetries.class) final int maxNumberOfRetries,
+                          @Parameter(HTTPStatusAlarmInterval.class) final int alarmInterval) {
+    this.clock = clock;
+    this.maxNumberOfRetries = maxNumberOfRetries;
+    this.alarmInterval = alarmInterval;
+    scheduleAlarm();
+  }
+
+  @Override
+  public String getUriSpecification() {
+    return uriSpecification;
+  }
+
+  @Override
+  public void setUriSpecification(final String newUriSpecification) {
+    this.uriSpecification = newUriSpecification;
+  }
+
+  @Override
+  public void onHttpRequest(final ParsedHttpRequest parsedHttpRequest, final HttpServletResponse response)
+      throws IOException, ServletException {
+    try (final PrintWriter writer = response.getWriter()) {
+      writer.write(waitAndGetMessage());
+      this.wasCalledViaHTTP = true;
+    }
+  }
+
+  @Override
+  public void onNext(final ReefServiceProtos.JobStatusProto value) {
+    LOG.log(Level.INFO, "Received status: {0}", value.getState().name());
+    // Record the status received and notify the thread to send an answer.
+    synchronized (this) {
+      this.statusMessagesToSend.add(value);
+      this.lastStatus = value;
+      this.notifyAll();
+    }
+  }
+
+  @Override
+  public ReefServiceProtos.JobStatusProto getLastStatus() {
+    return this.lastStatus;
+  }
+
+  @Override
+  public String toString() {
+    return "DriverStatusHTTPHandler{uriSpec=" + getUriSpecification() + "}";
+  }
+
+  /**
+   * Waits for a status message to be available and returns it.
+   *
+   * @return the first available status message.
+   */
+  String waitAndGetMessage() {
+    synchronized (this) {
+      // Wait for a message to send.
+      while (this.statusMessagesToSend.isEmpty()) {
+        try {
+          this.wait();
+        } catch (final InterruptedException e) {
+          LOG.log(Level.FINE, "Interrupted. Ignoring.");
+        }
+      }
+
+      // Send the message
+      return getMessageForStatus(this.statusMessagesToSend.poll());
+    }
+  }
+
+  /**
+   * Generates a string to be sent to the client based on a
+   * {@link ReefServiceProtos.JobStatusProto}.
+   *
+   * @param status the status to be converted to String.
+   * @return the string to be sent back to the HTTP client.
+   */
+  static String getMessageForStatus(final ReefServiceProtos.JobStatusProto status) {
+    return status.getState().name();
+  }
+
+  /**
+   * Schedules an alarm, if needed.
+   * <p>
+   * The alarm will prevent the Clock from going idle. This gives the .NET Client time to make a call to this HTTP
+   * handler.
+   */
+  private void scheduleAlarm() {
+    if (wasCalledViaHTTP || retry >= maxNumberOfRetries) {
+      // No alarm necessary anymore.
+      LOG.log(Level.INFO,
+          "Not scheduling additional alarms after {0} out of max {1} retries.",
+          new Object[] {retry, maxNumberOfRetries});
+      return;
+    }
+
+    // Scheduling an alarm will prevent the clock from going idle.
+    ++retry;
+    clock.scheduleAlarm(alarmInterval, alarmHandler);
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java
new file mode 100644
index 0000000..4d39bb3
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java
@@ -0,0 +1,976 @@
+/*
+ * 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.bridge.driver.service.grpc;
+
+import com.google.protobuf.ByteString;
+import io.grpc.*;
+import io.grpc.stub.StreamObserver;
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.common.grpc.GRPCUtils;
+import org.apache.reef.bridge.driver.common.grpc.ObserverCleanup;
+import org.apache.reef.bridge.driver.service.DriverClientException;
+import org.apache.reef.bridge.driver.service.DriverService;
+import org.apache.reef.bridge.service.parameters.DriverClientCommand;
+import org.apache.reef.bridge.proto.*;
+import org.apache.reef.bridge.proto.Void;
+import org.apache.reef.driver.context.*;
+import org.apache.reef.driver.evaluator.*;
+import org.apache.reef.driver.restart.DriverRestartCompleted;
+import org.apache.reef.driver.restart.DriverRestarted;
+import org.apache.reef.driver.task.*;
+import org.apache.reef.runtime.common.driver.context.EvaluatorContext;
+import org.apache.reef.runtime.common.driver.evaluator.AllocatedEvaluatorImpl;
+import org.apache.reef.runtime.common.driver.idle.IdleMessage;
+import org.apache.reef.runtime.common.files.REEFFileNames;
+import org.apache.reef.runtime.common.utils.ExceptionCodec;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.util.OSUtils;
+import org.apache.reef.util.Optional;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.remote.ports.TcpPortProvider;
+import org.apache.reef.wake.time.Clock;
+import org.apache.reef.wake.time.event.Alarm;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+
+import javax.inject.Inject;
+import java.io.*;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * GRPC DriverBridgeService that interacts with higher-level languages.
+ */
+@Private
+public final class GRPCDriverService implements DriverService {
+  private static final Logger LOG = Logger.getLogger(GRPCDriverService.class.getName());
+
+  private static final Void VOID = Void.newBuilder().build();
+
+  private Process driverProcess;
+
+  private enum StreamType { STDOUT, STDERR }
+
+  private Server server;
+
+  private DriverClientGrpc.DriverClientFutureStub clientStub;
+
+  private final Clock clock;
+
+  private final REEFFileNames reefFileNames;
+
+  private final ExceptionCodec exceptionCodec;
+
+  private final ConfigurationSerializer configurationSerializer;
+
+  private final EvaluatorRequestor evaluatorRequestor;
+
+  private final JVMProcessFactory jvmProcessFactory;
+
+  private final CLRProcessFactory clrProcessFactory;
+
+  private final TcpPortProvider tcpPortProvider;
+
+  private final String driverClientCommand;
+
+  private final Map<String, AllocatedEvaluator> allocatedEvaluatorMap = new HashMap<>();
+
+  private final Map<String, ActiveContext> activeContextMap = new HashMap<>();
+
+  private final Map<String, RunningTask> runningTaskMap = new HashMap<>();
+
+  private boolean stopped = false;
+
+  @Inject
+  private GRPCDriverService(
+      final Clock clock,
+      final REEFFileNames reefFileNames,
+      final EvaluatorRequestor evaluatorRequestor,
+      final ConfigurationSerializer configurationSerializer,
+      final JVMProcessFactory jvmProcessFactory,
+      final CLRProcessFactory clrProcessFactory,
+      final TcpPortProvider tcpPortProvider,
+      final ExceptionCodec exceptionCodec,
+      @Parameter(DriverClientCommand.class) final String driverClientCommand) {
+    this.clock = clock;
+    this.reefFileNames = reefFileNames;
+    this.exceptionCodec = exceptionCodec;
+    this.configurationSerializer = configurationSerializer;
+    this.jvmProcessFactory = jvmProcessFactory;
+    this.clrProcessFactory = clrProcessFactory;
+    this.evaluatorRequestor = evaluatorRequestor;
+    this.driverClientCommand = driverClientCommand;
+    this.tcpPortProvider = tcpPortProvider;
+  }
+
+  private void start() throws IOException, InterruptedException {
+    for (final int port : this.tcpPortProvider) {
+      try {
+        this.server = ServerBuilder.forPort(port)
+            .addService(new DriverBridgeServiceImpl())
+            .build()
+            .start();
+        LOG.log(Level.INFO, "Server started, listening on port [{0}]", port);
+        break;
+      } catch (final IOException e) {
+        LOG.log(Level.WARNING, "Unable to bind to port [{0}]", port);
+      }
+    }
+    if (this.server == null || this.server.isTerminated()) {
+      throw new IOException("Unable to start gRPC server");
+    }
+    final String cmd = this.driverClientCommand + " " + this.server.getPort();
+    final List<String> cmdOs = OSUtils.isWindows() ?
+        Arrays.asList("cmd.exe", "/c", cmd) : Arrays.asList("/bin/sh", "-c", cmd);
+    LOG.log(Level.INFO, "CMD: {0}", cmdOs);
+    this.driverProcess = new ProcessBuilder()
+        .command(cmdOs)
+        .redirectError(new File(this.reefFileNames.getDriverClientStderrFileName()))
+        .redirectOutput(new File(this.reefFileNames.getDriverClientStdoutFileName()))
+        .start();
+    synchronized (this) {
+      int attempts = 30; // give some time
+      /* wait for driver client process to register
+       * Note: attempts and wait time have been given reasonable hardcoded values for a driver
+       * client to register with the driver service (us). Making these values configurable would
+       * require additions to the ClientProtocol buffer such that they can be passed to the
+       * GRPCDriverServiceConfigurationProvider and bound to the appropriate NamedParameters. It
+       * is the opinion at the time of this writing that a driver client should be able to register
+       * within 10 seconds.
+       */
+      while (attempts-- > 0 && this.clientStub == null && driverProcessIsAlive()) {
+        LOG.log(Level.INFO, "waiting for driver process to register");
+        this.wait(1000); // a second
+      }
+    }
+    if (driverProcessIsAlive()) {
+      final Thread closeChildThread = new Thread() {
+        public void run() {
+          synchronized (GRPCDriverService.this) {
+            if (GRPCDriverService.this.driverProcess != null) {
+              GRPCDriverService.this.driverProcess.destroy();
+              GRPCDriverService.this.driverProcess = null;
+            }
+          }
+        }
+      };
+      // This is probably overkill since shutdown should be called in the stop handler.
+      Runtime.getRuntime().addShutdownHook(closeChildThread);
+    }
+  }
+
+  private void stop() {
+    stop(null);
+  }
+
+  private void stop(final Throwable t) {
+    LOG.log(Level.INFO, "STOP: gRPC Driver Service", t);
+    if (!stopped) {
+      try {
+        if (!clock.isClosed()) {
+          if (t != null) {
+            clock.stop(t);
+          } else {
+            clock.stop();
+          }
+        }
+        if (server != null) {
+          LOG.log(Level.INFO, "Shutdown gRPC");
+          this.server.shutdown();
+          this.server = null;
+        }
+        if (this.driverProcess != null) {
+          LOG.log(Level.INFO, "shutdown driver process");
+          dump();
+          this.driverProcess.destroy();
+          this.driverProcess = null;
+        }
+      } finally {
+        LOG.log(Level.INFO, "COMPLETED STOP: gRPC Driver Service");
+        stopped = true;
+      }
+    }
+  }
+
+  private void dump() {
+    if (!driverProcessIsAlive()) {
+      LOG.log(Level.INFO, "Exit code: {0}", this.driverProcess.exitValue());
+    }
+    dumpStream(StreamType.STDOUT);
+    dumpStream(StreamType.STDERR);
+  }
+
+  private void dumpStream(final StreamType type) {
+    final StringBuilder stringBuilder = new StringBuilder();
+
+    final String name;
+    final InputStream stream;
+    switch(type) {
+    case STDOUT:
+      name = "stdout";
+      stream = this.driverProcess.getInputStream();
+      break;
+    case STDERR:
+      name = "stderr";
+      stream = this.driverProcess.getErrorStream();
+      break;
+    default:
+      throw new RuntimeException("Invalid stream type value");
+    }
+
+    LOG.log(Level.INFO, "capturing driver process {0}", name);
+    try {
+      stringBuilder.append("\n==============================================\n");
+      try (final BufferedReader reader = new BufferedReader(new InputStreamReader(stream))) {
+        while (reader.ready()) {
+          stringBuilder.append(reader.readLine()).append('\n');
+        }
+      }
+      stringBuilder.append("\n==============================================\n");
+    } catch (final IOException e) {
+      LOG.log(Level.WARNING, "Error while capturing output stream", e);
+    }
+    LOG.log(Level.INFO, "{0}", stringBuilder);
+  }
+
+  /**
+   * Determines if the driver process is still alive by
+   * testing for its exit value, which throws {@link IllegalThreadStateException}
+   * if process is still running.
+   * @return true if driver process is alive, false otherwise
+   */
+  private boolean driverProcessIsAlive() {
+    if (this.driverProcess != null) {
+      try {
+        this.driverProcess.exitValue();
+      } catch (final IllegalThreadStateException e) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+
+  @Override
+  public IdleMessage getIdleStatus() {
+    final String componentName = "Java Bridge DriverService";
+    if (this.clientStub != null) {
+      try {
+        final IdleStatus idleStatus = this.clientStub.idlenessCheckHandler(VOID).get();
+        LOG.log(Level.INFO, "is idle: {0}", idleStatus.getIsIdle());
+        return new IdleMessage(
+            componentName,
+            idleStatus.getReason(),
+            idleStatus.getIsIdle());
+      } catch (final ExecutionException | InterruptedException e) {
+        stop(e);
+      }
+    }
+    return new IdleMessage(
+        componentName,
+        "stub not initialized",
+        true);
+  }
+
+  @Override
+  public void startHandler(final StartTime startTime) {
+    try {
+      start();
+      synchronized (this) {
+        if (this.clientStub != null) {
+          this.clientStub.startHandler(
+              StartTimeInfo.newBuilder().setStartTime(startTime.getTimestamp()).build());
+        } else {
+          stop(new IllegalStateException("Unable to start driver client"));
+        }
+      }
+    } catch (final IOException | InterruptedException e) {
+      stop(e);
+    }
+  }
+
+  @Override
+  public void stopHandler(final StopTime stopTime) {
+    synchronized (this) {
+      if (clientStub != null) {
+        final Future<ExceptionInfo> callCompletion = this.clientStub.stopHandler(
+            StopTimeInfo.newBuilder().setStopTime(stopTime.getTimestamp()).build());
+        try {
+          final ExceptionInfo error = callCompletion.get(5L, TimeUnit.MINUTES);
+          if (!error.getNoError()) {
+            final Optional<Throwable> t = parseException(error);
+            if (t.isPresent()) {
+              throw new RuntimeException("driver stop exception",
+                  t.get().getCause() != null ? t.get().getCause() : t.get());
+            } else {
+              throw new RuntimeException(error.getMessage() != null ? error.getMessage() : error.getName());
+            }
+          }
+        } catch (final TimeoutException e) {
+          throw new RuntimeException("stop handler timed out", e);
+        } catch (final InterruptedException | ExecutionException e) {
+          throw new RuntimeException("error in stop handler", e);
+        } finally {
+          stop();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void allocatedEvaluatorHandler(final AllocatedEvaluator eval) {
+    synchronized (this) {
+      this.allocatedEvaluatorMap.put(eval.getId(), eval);
+      this.clientStub.allocatedEvaluatorHandler(
+          EvaluatorInfo.newBuilder()
+              .setEvaluatorId(eval.getId())
+              .setDescriptorInfo(
+                  GRPCUtils.toEvaluatorDescriptorInfo(eval.getEvaluatorDescriptor()))
+              .build());
+    }
+  }
+
+  @Override
+  public void completedEvaluatorHandler(final CompletedEvaluator eval) {
+    synchronized (this) {
+      this.allocatedEvaluatorMap.remove(eval.getId());
+      this.clientStub.completedEvaluatorHandler(
+          EvaluatorInfo.newBuilder().setEvaluatorId(eval.getId()).build());
+    }
+  }
+
+  @Override
+  public void failedEvaluatorHandler(final FailedEvaluator eval) {
+    synchronized (this) {
+      this.allocatedEvaluatorMap.remove(eval.getId());
+      this.clientStub.failedEvaluatorHandler(
+          EvaluatorInfo.newBuilder().setEvaluatorId(eval.getId()).build());
+    }
+  }
+
+  @Override
+  public void activeContextHandler(final ActiveContext context) {
+    synchronized (this) {
+      this.activeContextMap.put(context.getId(), context);
+      this.clientStub.activeContextHandler(GRPCUtils.toContextInfo(context));
+    }
+  }
+
+  @Override
+  public void closedContextHandler(final ClosedContext context) {
+    synchronized (this) {
+      this.activeContextMap.remove(context.getId());
+      this.clientStub.closedContextHandler(GRPCUtils.toContextInfo(context));
+    }
+  }
+
+  @Override
+  public void failedContextHandler(final FailedContext context) {
+    synchronized (this) {
+      final ExceptionInfo error;
+      if (context.getReason().isPresent()) {
+        final Throwable reason = context.getReason().get();
+        error = GRPCUtils.createExceptionInfo(this.exceptionCodec, reason);
+      } else if (context.getData().isPresent()) {
+        error = ExceptionInfo.newBuilder()
+            .setName(context.toString())
+            .setMessage(context.getDescription().orElse(
+                context.getMessage() != null ? context.getMessage() : ""))
+            .setData(ByteString.copyFrom(context.getData().get()))
+            .build();
+      } else {
+        error = GRPCUtils.createExceptionInfo(this.exceptionCodec, context.asError());
+      }
+      this.activeContextMap.remove(context.getId());
+      this.clientStub.failedContextHandler(GRPCUtils.toContextInfo(context, error));
+    }
+  }
+
+  @Override
+  public void contextMessageHandler(final ContextMessage message) {
+    synchronized (this) {
+      this.clientStub.contextMessageHandler(
+          ContextMessageInfo.newBuilder()
+              .setContextId(message.getId())
+              .setMessageSourceId(message.getMessageSourceID())
+              .setSequenceNumber(message.getSequenceNumber())
+              .setPayload(ByteString.copyFrom(message.get()))
+              .build());
+    }
+  }
+
+  @Override
+  public void runningTaskHandler(final RunningTask task) {
+    synchronized (this) {
+      final ActiveContext context = task.getActiveContext();
+      if (!this.activeContextMap.containsKey(context.getId())) {
+        this.activeContextMap.put(context.getId(), context);
+      }
+      this.runningTaskMap.put(task.getId(), task);
+      this.clientStub.runningTaskHandler(
+          TaskInfo.newBuilder()
+              .setTaskId(task.getId())
+              .setContext(GRPCUtils.toContextInfo(context))
+              .build());
+    }
+  }
+
+  @Override
+  public void failedTaskHandler(final FailedTask task) {
+    synchronized (this) {
+      if (task.getActiveContext().isPresent() &&
+          !this.activeContextMap.containsKey(task.getActiveContext().get().getId())) {
+        this.activeContextMap.put(task.getActiveContext().get().getId(), task.getActiveContext().get());
+      }
+      final TaskInfo.Builder taskInfoBuilder = TaskInfo.newBuilder()
+          .setTaskId(task.getId());
+      if (task.getActiveContext().isPresent()) {
+        taskInfoBuilder.setContext(GRPCUtils.toContextInfo(task.getActiveContext().get()));
+      }
+      if (task.getReason().isPresent()) {
+        taskInfoBuilder.setException(GRPCUtils.createExceptionInfo(this.exceptionCodec, task.getReason().get()));
+      } else if (task.getData().isPresent()) {
+        final Throwable reason = task.asError();
+        taskInfoBuilder.setException(ExceptionInfo.newBuilder()
+            .setName(reason.toString())
+            .setMessage(task.getMessage() != null ? task.getMessage() : "")
+            .setData(ByteString.copyFrom(task.getData().get()))
+            .build());
+      } else {
+        taskInfoBuilder.setException(GRPCUtils.createExceptionInfo(this.exceptionCodec, task.asError()));
+      }
+      this.runningTaskMap.remove(task.getId());
+      this.clientStub.failedTaskHandler(taskInfoBuilder.build());
+    }
+  }
+
+  @Override
+  public void completedTaskHandler(final CompletedTask task) {
+    synchronized (this) {
+      if (!this.activeContextMap.containsKey(task.getActiveContext().getId())) {
+        this.activeContextMap.put(task.getActiveContext().getId(), task.getActiveContext());
+      }
+      this.runningTaskMap.remove(task.getId());
+      this.clientStub.completedTaskHandler(
+          TaskInfo.newBuilder()
+              .setTaskId(task.getId())
+              .setContext(GRPCUtils.toContextInfo(task.getActiveContext()))
+              .build());
+    }
+  }
+
+  @Override
+  public void suspendedTaskHandler(final SuspendedTask task) {
+    synchronized (this) {
+      if (!this.activeContextMap.containsKey(task.getActiveContext().getId())) {
+        this.activeContextMap.put(task.getActiveContext().getId(), task.getActiveContext());
+      }
+      this.runningTaskMap.remove(task.getId());
+      this.clientStub.suspendedTaskHandler(
+          TaskInfo.newBuilder()
+              .setTaskId(task.getId())
+              .setContext(GRPCUtils.toContextInfo(task.getActiveContext()))
+              .setResult(task.get() == null || task.get().length == 0 ?
+                  null : ByteString.copyFrom(task.get()))
+              .build());
+    }
+  }
+
+  @Override
+  public void taskMessageHandler(final TaskMessage message) {
+    synchronized (this) {
+      this.clientStub.taskMessageHandler(
+          TaskMessageInfo.newBuilder()
+              .setTaskId(message.getId())
+              .setContextId(message.getContextId())
+              .setMessageSourceId(message.getMessageSourceID())
+              .setSequenceNumber(message.getSequenceNumber())
+              .setPayload(ByteString.copyFrom(message.get()))
+              .build());
+    }
+  }
+
+  @Override
+  public void clientMessageHandler(final byte[] message) {
+    synchronized (this) {
+      this.clientStub.clientMessageHandler(
+          ClientMessageInfo.newBuilder()
+              .setPayload(ByteString.copyFrom(message))
+              .build());
+    }
+  }
+
+  @Override
+  public void clientCloseHandler() {
+    synchronized (this) {
+      this.clientStub.clientCloseHandler(VOID);
+    }
+  }
+
+  @Override
+  public void clientCloseWithMessageHandler(final byte[] message) {
+    synchronized (this) {
+      this.clientStub.clientCloseWithMessageHandler(
+          ClientMessageInfo.newBuilder()
+              .setPayload(ByteString.copyFrom(message))
+              .build());
+    }
+  }
+
+  @Override
+  public void driverRestarted(final DriverRestarted restart) {
+    try {
+      start();
+      synchronized (this) {
+        if (this.clientStub != null) {
+          this.clientStub.driverRestartHandler(DriverRestartInfo.newBuilder()
+              .setResubmissionAttempts(restart.getResubmissionAttempts())
+              .setStartTime(StartTimeInfo.newBuilder()
+                  .setStartTime(restart.getStartTime().getTimestamp()).build())
+              .addAllExpectedEvaluatorIds(restart.getExpectedEvaluatorIds())
+              .build());
+        } else {
+          stop(new DriverClientException("Failed to restart driver client"));
+        }
+      }
+    } catch (final InterruptedException | IOException e) {
+      stop(e);
+    }
+  }
+
+  @Override
+  public void restartRunningTask(final RunningTask task) {
+    synchronized (this) {
+      final ActiveContext context = task.getActiveContext();
+      if (!this.activeContextMap.containsKey(context.getId())) {
+        this.activeContextMap.put(context.getId(), context);
+      }
+      this.runningTaskMap.put(task.getId(), task);
+      this.clientStub.driverRestartRunningTaskHandler(
+          TaskInfo.newBuilder()
+              .setTaskId(task.getId())
+              .setContext(GRPCUtils.toContextInfo(context))
+              .build());
+    }
+  }
+
+  @Override
+  public void restartActiveContext(final ActiveContext context) {
+    synchronized (this) {
+      this.activeContextMap.put(context.getId(), context);
+      this.clientStub.driverRestartActiveContextHandler(
+          GRPCUtils.toContextInfo(context));
+    }
+  }
+
+  @Override
+  public void driverRestartCompleted(final DriverRestartCompleted restartCompleted) {
+    synchronized (this) {
+      this.clientStub.driverRestartCompletedHandler(DriverRestartCompletedInfo.newBuilder()
+          .setCompletionTime(StopTimeInfo.newBuilder()
+              .setStopTime(restartCompleted.getCompletedTime().getTimestamp()).build())
+          .setIsTimedOut(restartCompleted.isTimedOut())
+          .build());
+    }
+  }
+
+  @Override
+  public void restartFailedEvalautor(final FailedEvaluator evaluator) {
+    synchronized (this) {
+      this.clientStub.driverRestartFailedEvaluatorHandler(EvaluatorInfo.newBuilder()
+          .setEvaluatorId(evaluator.getId())
+          .setFailure(EvaluatorInfo.FailureInfo.newBuilder()
+              .setMessage(evaluator.getEvaluatorException() != null ?
+                  evaluator.getEvaluatorException().getMessage() : "unknown failure during restart")
+              .build())
+          .build());
+    }
+  }
+
+  private Optional<Throwable> parseException(final ExceptionInfo info) {
+    if (info.getData() == null || info.getData().isEmpty()) {
+      return Optional.empty();
+    } else {
+      return exceptionCodec.fromBytes(info.getData().toByteArray());
+    }
+  }
+
+  private final class DriverBridgeServiceImpl
+      extends DriverServiceGrpc.DriverServiceImplBase {
+
+    @Override
+    public void registerDriverClient(
+        final DriverClientRegistration request,
+        final StreamObserver<Void> responseObserver) {
+      LOG.log(Level.INFO, "driver client register");
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        if (request.hasException()) {
+          LOG.log(Level.SEVERE, "Driver client initialization exception");
+          final Optional<Throwable> ex = parseException(request.getException());
+          if (ex.isPresent()) {
+            GRPCDriverService.this.clock.stop(ex.get());
+          } else {
+            GRPCDriverService.this.clock.stop(new RuntimeException(
+                request.getException().getMessage() == null ?
+                    request.getException().getName() :
+                    request.getException().getMessage()
+            ));
+          }
+        } else {
+          final ManagedChannel channel = ManagedChannelBuilder
+              .forAddress(request.getHost(), request.getPort())
+              .usePlaintext()
+              .build();
+          synchronized (GRPCDriverService.this) {
+            GRPCDriverService.this.clientStub = DriverClientGrpc.newFutureStub(channel);
+            GRPCDriverService.this.notifyAll();
+          }
+          LOG.log(Level.INFO, "Driver has registered on port {0}", request.getPort());
+        }
+      }
+    }
+
+    @Override
+    public void requestResources(
+        final ResourceRequest request,
+        final StreamObserver<Void> responseObserver) {
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        synchronized (GRPCDriverService.this) {
+          EvaluatorRequest.Builder requestBuilder = GRPCDriverService.this.evaluatorRequestor.newRequest();
+          requestBuilder.setNumber(request.getResourceCount());
+          requestBuilder.setNumberOfCores(request.getCores());
+          requestBuilder.setMemory(request.getMemorySize());
+          requestBuilder.setRelaxLocality(request.getRelaxLocality());
+          requestBuilder.setRuntimeName(request.getRuntimeName());
+          if (request.getNodeNameListCount() > 0) {
+            requestBuilder.addNodeNames(request.getNodeNameListList());
+          }
+          if (request.getRackNameListCount() > 0) {
+            for (final String rackName : request.getRackNameListList()) {
+              requestBuilder.addRackName(rackName);
+            }
+          }
+          GRPCDriverService.this.evaluatorRequestor.submit(requestBuilder.build());
+        }
+      }
+    }
+
+    @Override
+    public void shutdown(
+        final ShutdownRequest request,
+        final StreamObserver<Void> responseObserver) {
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        LOG.log(Level.INFO, "driver shutdown");
+        if (request.hasException()) {
+          final Optional<Throwable> exception = parseException(request.getException());
+          if (exception.isPresent()) {
+            LOG.log(Level.INFO, "driver exception", exception.get());
+            GRPCDriverService.this.clock.stop(exception.get());
+          } else {
+            // exception that cannot be parsed in java
+            GRPCDriverService.this.clock.stop(
+                new DriverClientException(request.getException().getMessage()));
+          }
+        } else {
+          LOG.log(Level.INFO, "clean shutdown");
+          GRPCDriverService.this.clock.stop();
+        }
+      }
+    }
+
+    @Override
+    public void setAlarm(
+        final AlarmRequest request,
+        final StreamObserver<Void> responseObserver) {
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        // do not synchronize when scheduling an alarm (or deadlock)
+        LOG.log(Level.INFO, "Set alarm {0} offset {1}",
+            new Object[] {request.getAlarmId(), request.getTimeoutMs()});
+        LOG.log(Level.INFO, "Alarm class {0}", GRPCDriverService.this.clock.getClass());
+        GRPCDriverService.this.clock.scheduleAlarm(request.getTimeoutMs(), new EventHandler<Alarm>() {
+          @Override
+          public void onNext(final Alarm value) {
+            LOG.log(Level.INFO, "Trigger alarm {0}", request.getAlarmId());
+            synchronized (GRPCDriverService.this) {
+              GRPCDriverService.this.clientStub.alarmTrigger(
+                  AlarmTriggerInfo.newBuilder().setAlarmId(request.getAlarmId()).build());
+              LOG.log(Level.INFO, "DONE: trigger alarm {0}", request.getAlarmId());
+            }
+          }
+        });
+        LOG.log(Level.INFO, "Alarm {0} scheduled is idle? {1}",
+            new Object[] {request.getAlarmId(), clock.isIdle()});
+      }
+    }
+
+    @Override
+    public void allocatedEvaluatorOp(
+        final AllocatedEvaluatorRequest request,
+        final StreamObserver<Void> responseObserver) {
+      try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+        if (request.getEvaluatorConfiguration() == null) {
+          responseObserver.onError(Status.INTERNAL
+              .withDescription("Evaluator configuration required")
+              .asRuntimeException());
+        } else if (request.getContextConfiguration() == null && request.getTaskConfiguration() == null) {
+          responseObserver.onError(Status.INTERNAL
+              .withDescription("Context and/or Task configuration required")
+              .asRuntimeException());
+        } else {
+          synchronized (GRPCDriverService.this) {
+            if (!GRPCDriverService.this.allocatedEvaluatorMap.containsKey(request.getEvaluatorId())) {
+              responseObserver.onError(Status.INTERNAL
+                  .withDescription("Unknown allocated evaluator " + request.getEvaluatorId())
+                  .asRuntimeException());
+            }
+            final AllocatedEvaluator evaluator =
+                GRPCDriverService.this.allocatedEvaluatorMap.get(request.getEvaluatorId());
+            if (request.getCloseEvaluator()) {
+              evaluator.close();
+            } else {
+              for (final String file : request.getAddFilesList()) {
+                evaluator.addFile(new File(file));
+              }
+              for (final String library : request.getAddLibrariesList()) {
+                evaluator.addLibrary(new File(library));
+              }
+              if (request.getSetProcess() != null) {
+                final AllocatedEvaluatorRequest.EvaluatorProcessRequest processRequest =
+                    request.getSetProcess();
+                switch (evaluator.getEvaluatorDescriptor().getProcess().getType()) {
+                case JVM:
+                  setJVMProcess(evaluator, processRequest);
+                  break;
+                case CLR:
+                  setCLRProcess(evaluator, processRequest);
+                  break;
+                default:
+                  throw new RuntimeException("Unknown evaluator process type");
+                }
+              }
+              if (StringUtils.isEmpty(request.getEvaluatorConfiguration())) {
+                // Assume that we are running Java driver client, but this assumption could be a bug so log a warning
+                LOG.log(Level.WARNING, "No evaluator configuration detected. Assuming a Java driver client.");
+                if (StringUtils.isNotEmpty(request.getContextConfiguration()) &&
+                    StringUtils.isNotEmpty(request.getTaskConfiguration())) {
+                  // submit context and task
+                  try {
+                    evaluator.submitContextAndTask(
+                        configurationSerializer.fromString(request.getContextConfiguration()),
+                        configurationSerializer.fromString(request.getTaskConfiguration()));
+                  } catch (final IOException e) {
+                    throw new RuntimeException("error submitting task and context", e);
+                  }
+                } else if (StringUtils.isNotEmpty(request.getContextConfiguration())) {
+                  // submit context
+                  try {
+                    evaluator.submitContext(configurationSerializer.fromString(request.getContextConfiguration()));
+                  } catch (final IOException e) {
+                    throw new RuntimeException("error submitting context", e);
+                  }
+                } else if (StringUtils.isNotEmpty(request.getTaskConfiguration())) {
+                  // submit task
+                  try {
+                    evaluator.submitTask(configurationSerializer.fromString(request.getTaskConfiguration()));
+                  } catch (final IOException e) {
+                    throw new RuntimeException("error submitting task", e);
+                  }
+                } else {
+                  throw new RuntimeException("Missing check for required evaluator configurations");
+                }
+              } else {
+                if (StringUtils.isNotEmpty(request.getContextConfiguration()) &&
+                    StringUtils.isNotEmpty(request.getTaskConfiguration())) {
+                  // submit context and task
+                  ((AllocatedEvaluatorImpl) evaluator).submitContextAndTask(
+                      request.getEvaluatorConfiguration(),
+                      request.getContextConfiguration(),
+                      request.getTaskConfiguration());
+                } else if (StringUtils.isNotEmpty(request.getContextConfiguration())) {
+                  // submit context
+                  ((AllocatedEvaluatorImpl) evaluator).submitContext(
+                      request.getEvaluatorConfiguration(),
+                      request.getContextConfiguration());
+                } else if (StringUtils.isNotEmpty(request.getTaskConfiguration())) {
+                  // submit task
+                  ((AllocatedEvaluatorImpl) evaluator).submitTask(
+                      request.getEvaluatorConfiguration(),
+                      request.getTaskConfiguration());
+                } else {
+                  throw new RuntimeException("Missing check for required evaluator configurations");
+                }
+              }
+            }
+          }
+        }
+      }
+    }
+
+    @Override
+    public void activeContextOp(
+        final ActiveContextRequest request,
+        final StreamObserver<Void> responseObserver) {
+      synchronized (GRPCDriverService.this) {
+        final String contextId = request.getContextId();
+        final ActiveContext context = GRPCDriverService.this.activeContextMap.get(contextId);
+        if (context == null) {
+          LOG.log(Level.SEVERE, "Context does not exist with id {0}", contextId);
+          responseObserver.onError(Status.INTERNAL
+              .withDescription("Context does not exist with id " + contextId)
+              .asRuntimeException());
+          return;
+        }
+        switch (request.getOperationCase()) {
+        case CLOSE_CONTEXT:
+          if (request.getCloseContext()) {
+            try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+              LOG.log(Level.INFO, "closing context {0}", context.getId());
+              context.close();
+            }
+          } else {
+            LOG.log(Level.SEVERE, "Close context operation not set to true");
+            responseObserver.onError(Status.INTERNAL
+                .withDescription("Close context operation not set to true")
+                .asRuntimeException());
+          }
+          break;
+        case MESSAGE:
+          if (request.getMessage() != null) {
+            try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+              LOG.log(Level.INFO, "send message to context {0}", context.getId());
+              context.sendMessage(request.getMessage().toByteArray());
+            }
+          } else {
+            responseObserver.onError(Status.INTERNAL
+                .withDescription("Empty message on operation send message").asRuntimeException());
+          }
+          break;
+        case NEW_CONTEXT_REQUEST:
+          try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+            LOG.log(Level.INFO, "submitting child context to context {0}", context.getId());
+            ((EvaluatorContext) context).submitContext(request.getNewContextRequest());
+          }
+          break;
+        case NEW_TASK_REQUEST:
+          try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+            LOG.log(Level.INFO, "submitting task to context {0}", context.getId());
+            ((EvaluatorContext) context).submitTask(request.getNewTaskRequest());
+          }
+          break;
+        default:
+          throw new RuntimeException("Unknown operation " + request.getOperationCase());
+        }
+      }
+    }
+
+    @Override
+    public void runningTaskOp(
+        final RunningTaskRequest request,
+        final StreamObserver<Void> responseObserver) {
+      synchronized (GRPCDriverService.this) {
+        if (!GRPCDriverService.this.runningTaskMap.containsKey(request.getTaskId())) {
+          LOG.log(Level.WARNING, "Unknown task id {0}", request.getTaskId());
+          responseObserver.onError(Status.INTERNAL
+              .withDescription("Task does not exist with id " + request.getTaskId()).asRuntimeException());
+        } else {
+          try (final ObserverCleanup _cleanup = ObserverCleanup.of(responseObserver)) {
+            final RunningTask task = GRPCDriverService.this.runningTaskMap.get(request.getTaskId());
+            switch (request.getOperation()) {
+            case CLOSE:
+              LOG.log(Level.INFO, "close task {0}", task.getId());
+              if (request.getMessage().isEmpty()) {
+                task.close();
+              } else {
+                task.close(request.getMessage().toByteArray());
+              }
+              break;
+            case SUSPEND:
+              LOG.log(Level.INFO, "suspend task {0}", task.getId());
+              if (request.getMessage().isEmpty()) {
+                task.suspend();
+              } else {
+                task.suspend(request.getMessage().toByteArray());
+              }
+              break;
+            case SEND_MESSAGE:
+              LOG.log(Level.INFO, "send message to task {0}", task.getId());
+              task.send(request.getMessage().toByteArray());
+              break;
+            default:
+              throw new RuntimeException("Unknown operation " + request.getOperation());
+            }
+          }
+        }
+      }
+    }
+
+    private void setCLRProcess(
+        final AllocatedEvaluator evaluator,
+        final AllocatedEvaluatorRequest.EvaluatorProcessRequest processRequest) {
+      final CLRProcess process = GRPCDriverService.this.clrProcessFactory.newEvaluatorProcess();
+      if (processRequest.getMemoryMb() > 0) {
+        process.setMemory(processRequest.getMemoryMb());
+      }
+      if (processRequest.getConfigurationFileName() != null) {
+        process.setConfigurationFileName(processRequest.getConfigurationFileName());
+      }
+      if (processRequest.getStandardOut() != null) {
+        process.setStandardOut(processRequest.getStandardOut());
+      }
+      if (processRequest.getStandardErr() != null) {
+        process.setStandardErr(processRequest.getStandardErr());
+      }
+      evaluator.setProcess(process);
+    }
+
+    private void setJVMProcess(
+        final AllocatedEvaluator evaluator,
+        final AllocatedEvaluatorRequest.EvaluatorProcessRequest processRequest) {
+      final JVMProcess process = GRPCDriverService.this.jvmProcessFactory.newEvaluatorProcess();
+      if (processRequest.getMemoryMb() > 0) {
+        process.setMemory(processRequest.getMemoryMb());
+      }
+      if (processRequest.getConfigurationFileName() != null) {
+        process.setConfigurationFileName(processRequest.getConfigurationFileName());
+      }
+      if (processRequest.getStandardOut() != null) {
+        process.setStandardOut(processRequest.getStandardOut());
+      }
+      if (processRequest.getStandardErr() != null) {
+        process.setStandardErr(processRequest.getStandardErr());
+      }
+      if (processRequest.getOptionsCount() > 0) {
+        for (final String option : processRequest.getOptionsList()) {
+          process.addOption(option);
+        }
+      }
+      evaluator.setProcess(process);
+    }
+  }
+}
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverServiceConfigurationProvider.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverServiceConfigurationProvider.java
new file mode 100644
index 0000000..34ea463
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverServiceConfigurationProvider.java
@@ -0,0 +1,156 @@
+/*
+ * 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.bridge.driver.service.grpc;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.service.DriverServiceConfiguration;
+import org.apache.reef.bridge.driver.service.DriverServiceHandlers;
+import org.apache.reef.bridge.driver.service.DriverServiceConfigurationProvider;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.DriverConfiguration;
+import org.apache.reef.client.DriverRestartConfiguration;
+import org.apache.reef.client.parameters.DriverConfigurationProviders;
+import org.apache.reef.io.TcpPortConfigurationProvider;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.formats.ConfigurationModule;
+import org.apache.reef.util.EnvironmentUtils;
+import org.apache.reef.wake.remote.ports.parameters.TcpPortRangeBegin;
+import org.apache.reef.wake.remote.ports.parameters.TcpPortRangeCount;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * GRPC driver service configuration provider.
+ */
+@Private
+public final class GRPCDriverServiceConfigurationProvider implements DriverServiceConfigurationProvider {
+
+  @Inject
+  private GRPCDriverServiceConfigurationProvider() {
+  }
+
+  @Override
+  public Configuration getDriverServiceConfiguration(
+      final ClientProtocol.DriverClientConfiguration driverConfiguration) {
+    Configuration driverServiceConfiguration = DriverServiceConfiguration.CONF
+        .set(DriverServiceConfiguration.DRIVER_SERVICE_IMPL, GRPCDriverService.class)
+        .set(DriverServiceConfiguration.DRIVER_CLIENT_COMMAND, driverConfiguration.getDriverClientLaunchCommand())
+        .build();
+    return driverConfiguration.getDriverRestartEnable() ?
+        Configurations.merge(
+            driverServiceConfiguration,
+            getDriverRestartConfiguration(driverConfiguration),
+            getDriverConfiguration(driverConfiguration),
+            getTcpPortRangeConfiguration(driverConfiguration)) :
+        Configurations.merge(
+            driverServiceConfiguration,
+            getDriverConfiguration(driverConfiguration),
+            getTcpPortRangeConfiguration(driverConfiguration));
+  }
+
+  private Configuration getTcpPortRangeConfiguration(
+      final ClientProtocol.DriverClientConfiguration driverClientConfigurationProto) {
+    JavaConfigurationBuilder configurationModuleBuilder = Tang.Factory.getTang().newConfigurationBuilder()
+        .bindSetEntry(DriverConfigurationProviders.class, TcpPortConfigurationProvider.class);
+    // Setup TCP constraints
+    if (driverClientConfigurationProto.getTcpPortRangeBegin() > 0) {
+      configurationModuleBuilder = configurationModuleBuilder
+          .bindNamedParameter(TcpPortRangeBegin.class,
+              Integer.toString(driverClientConfigurationProto.getTcpPortRangeBegin()));
+    }
+    if (driverClientConfigurationProto.getTcpPortRangeCount() > 0) {
+      configurationModuleBuilder = configurationModuleBuilder
+          .bindNamedParameter(TcpPortRangeCount.class,
+              Integer.toString(driverClientConfigurationProto.getTcpPortRangeCount()));
+    }
+    if (driverClientConfigurationProto.getTcpPortRangeTryCount() > 0) {
+      configurationModuleBuilder = configurationModuleBuilder
+          .bindNamedParameter(TcpPortRangeCount.class,
+              Integer.toString(driverClientConfigurationProto.getTcpPortRangeTryCount()));
+    }
+    return configurationModuleBuilder.build();
+  }
+
+  private Configuration getDriverConfiguration(
+      final ClientProtocol.DriverClientConfiguration driverConfiguration) {
+    ConfigurationModule driverServiceConfigurationModule = DriverServiceConfiguration.STATIC_DRIVER_CONF_MODULE
+        .set(DriverConfiguration.DRIVER_IDENTIFIER, driverConfiguration.getJobid());
+
+    // Set file dependencies
+    final List<String> localLibraries = new ArrayList<>();
+    localLibraries.add(EnvironmentUtils.getClassLocation(GRPCDriverService.class));
+    localLibraries.addAll(driverConfiguration.getLocalLibrariesList());
+    driverServiceConfigurationModule = driverServiceConfigurationModule
+        .setMultiple(DriverConfiguration.LOCAL_LIBRARIES, localLibraries);
+    driverServiceConfigurationModule = driverServiceConfigurationModule
+        .setMultiple(DriverConfiguration.GLOBAL_LIBRARIES,
+            driverConfiguration.getGlobalLibrariesList());
+    driverServiceConfigurationModule = driverServiceConfigurationModule
+        .setMultiple(DriverConfiguration.LOCAL_FILES,
+            driverConfiguration.getLocalFilesList());
+    driverServiceConfigurationModule = driverServiceConfigurationModule
+        .setMultiple(DriverConfiguration.GLOBAL_FILES,
+            driverConfiguration.getGlobalFilesList());
+    // Setup driver resources
+    if (driverConfiguration.getCpuCores() > 0) {
+      driverServiceConfigurationModule = driverServiceConfigurationModule
+          .set(DriverConfiguration.DRIVER_CPU_CORES, driverConfiguration.getCpuCores());
+    }
+    if (driverConfiguration.getMemoryMb() > 0) {
+      driverServiceConfigurationModule = driverServiceConfigurationModule
+          .set(DriverConfiguration.DRIVER_MEMORY, driverConfiguration.getMemoryMb());
+    }
+    // Job submission directory
+    if (StringUtils.isNotEmpty(driverConfiguration.getDriverJobSubmissionDirectory())) {
+      driverServiceConfigurationModule = driverServiceConfigurationModule
+          .set(DriverConfiguration.DRIVER_JOB_SUBMISSION_DIRECTORY,
+              driverConfiguration.getDriverJobSubmissionDirectory());
+    }
+    return !driverConfiguration.getEnableHttpDriver() ? driverServiceConfigurationModule.build() :
+        Configurations.merge(DriverServiceConfiguration.HTTP_AND_NAMESERVER, driverServiceConfigurationModule.build());
+  }
+
+  private Configuration getDriverRestartConfiguration(
+      final ClientProtocol.DriverClientConfiguration driverConfiguration) {
+    final ConfigurationModule restartConfModule = DriverRestartConfiguration.CONF
+        .set(DriverRestartConfiguration.ON_DRIVER_RESTARTED,
+            DriverServiceHandlers.DriverRestartHandler.class)
+        .set(DriverRestartConfiguration.ON_DRIVER_RESTART_CONTEXT_ACTIVE,
+            DriverServiceHandlers.DriverRestartActiveContextHandler.class)
+        .set(DriverRestartConfiguration.ON_DRIVER_RESTART_TASK_RUNNING,
+            DriverServiceHandlers.DriverRestartRunningTaskHandler.class)
+        .set(DriverRestartConfiguration.ON_DRIVER_RESTART_COMPLETED,
+            DriverServiceHandlers.DriverRestartCompletedHandler.class)
+        .set(DriverRestartConfiguration.ON_DRIVER_RESTART_EVALUATOR_FAILED,
+            DriverServiceHandlers.DriverRestartFailedEvaluatorHandler.class);
+    return driverConfiguration.getDriverRestartEvaluatorRecoverySeconds() > 0 ?
+        restartConfModule
+            .set(DriverRestartConfiguration.DRIVER_RESTART_EVALUATOR_RECOVERY_SECONDS,
+                driverConfiguration.getDriverRestartEvaluatorRecoverySeconds())
+            .build() :
+        restartConfModule.build();
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/package-info.java
index 9967d9b..a94328d 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * GRPC implementation for driver bridge service.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.service.grpc;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/package-info.java
index 9967d9b..9b58cb0 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * The Java-side of the CLR/Java bridge interop via gRPC/Protocol Buffers.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.driver.service;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/DriverClientCommand.java
similarity index 68%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/DriverClientCommand.java
index 9967d9b..255f60d 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/DriverClientCommand.java
@@ -16,20 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+package org.apache.reef.bridge.service.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * What command to use when starting bridge process.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@NamedParameter(doc = "The command to launch bridge driver process",
+    short_name = "command")
+public final class DriverClientCommand implements Name<String> {
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/HTTPStatusAlarmInterval.java
similarity index 63%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/HTTPStatusAlarmInterval.java
index 9967d9b..7c013b2 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/HTTPStatusAlarmInterval.java
@@ -16,20 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
+package org.apache.reef.bridge.driver.service.parameters;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * The interval between alarms in DriverStatusHTTPHandler.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@NamedParameter(default_value = "200", doc = "The interval between alarms in DriverStatusHTTPHandler.")
+public final class HTTPStatusAlarmInterval implements Name<Integer> {
+
+  private HTTPStatusAlarmInterval() {
+    //intentionally empty
+  }
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/HTTPStatusNumberOfRetries.java
similarity index 63%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/HTTPStatusNumberOfRetries.java
index 9967d9b..c3d16be 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/HTTPStatusNumberOfRetries.java
@@ -16,20 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
+package org.apache.reef.bridge.driver.service.parameters;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
 
 /**
- * Test suite of tests covering failure scenarios.
+ * Number of times the HTTPStatusHandler will advance its alarm.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
+@NamedParameter(default_value = "10", doc = "Number of times the HTTPStatusHandler will advance its alarm.")
+public final class HTTPStatusNumberOfRetries implements Name<Integer> {
+
+  private HTTPStatusNumberOfRetries() {
+    // Intentionally empty.
+  }
 }
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/package-info.java
index 9967d9b..6a3b956 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/parameters/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Driver bridge service parameters.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.service.parameters;
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java
new file mode 100644
index 0000000..858d6af
--- /dev/null
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java
@@ -0,0 +1,71 @@
+/*
+ * 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.bridge.examples.hello;
+
+import org.apache.reef.bridge.driver.client.DriverClientConfiguration;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.bridge.client.DriverServiceLauncher;
+import org.apache.reef.examples.hello.HelloDriver;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.util.EnvironmentUtils;
+import org.apache.reef.util.ThreadLogger;
+
+import java.io.IOException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * The Client for Hello REEF example.
+ */
+public final class HelloREEF {
+
+  private static final Logger LOG = Logger.getLogger(HelloREEF.class.getName());
+
+  /** Configuration of the HelloREEF driver. */
+  private static final Configuration DRIVER_CONFIG =
+      DriverClientConfiguration.CONF
+          .set(DriverClientConfiguration.ON_DRIVER_STARTED, HelloDriver.StartHandler.class)
+          .set(DriverClientConfiguration.ON_EVALUATOR_ALLOCATED, HelloDriver.EvaluatorAllocatedHandler.class)
+          .build();
+
+  /**
+   * Start Hello REEF job with local runtime.
+   * @param args command line parameters.
+   * @throws InjectionException configuration error.
+   */
+  public static void main(final String[] args) throws InjectionException, IOException {
+
+    final ClientProtocol.DriverClientConfiguration.Builder builder =
+        ClientProtocol.DriverClientConfiguration.newBuilder();
+    builder.setJobid("HelloREEF");
+    builder.setEnableHttpDriver(false);
+    builder.setOperatingSystem(ClientProtocol.DriverClientConfiguration.OS.LINUX);
+    builder.setAzbatchRuntime(ClientProtocol.AzureBatchRuntimeParameters.newBuilder()
+        .build());
+    builder.addGlobalLibraries(EnvironmentUtils.getClassLocation(HelloDriver.class));
+
+    DriverServiceLauncher.submit(builder.build(), DRIVER_CONFIG);
+    LOG.log(Level.INFO, "REEF job completed");
+    ThreadLogger.logThreads(LOG, Level.FINE, "Threads running at the end of HelloREEF:");
+  }
+
+  /** Empty private constructor to prohibit instantiation of utility class. */
+  private HelloREEF() { }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/package-info.java
index 9967d9b..320e720 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Java bridge Hello REEF.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.examples.hello;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/package-info.java
index 9967d9b..a39bd45 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Java bridge examples.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.bridge.examples;
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/driver/evaluator/EvaluatorDescriptorImpl.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/driver/evaluator/EvaluatorDescriptorImpl.java
index f446855..e9627fb 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/driver/evaluator/EvaluatorDescriptorImpl.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/driver/evaluator/EvaluatorDescriptorImpl.java
@@ -31,7 +31,7 @@ import javax.inject.Inject;
  */
 @Private
 @DriverSide
-final class EvaluatorDescriptorImpl implements EvaluatorDescriptor {
+public final class EvaluatorDescriptorImpl implements EvaluatorDescriptor {
 
   private final NodeDescriptor nodeDescriptor;
   private final int megaBytes;
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java
index 811bbd4..e7b6818 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java
@@ -45,6 +45,8 @@ public final class REEFFileNames {
   private static final String EVALUATOR_FOLDER_PREFIX = "reef-evaluator-";
   private static final String DRIVER_STDERR = "driver.stderr";
   private static final String DRIVER_STDOUT = "driver.stdout";
+  private static final String DRIVER_CLIENT_STDERR = "driverclient.stderr";
+  private static final String DRIVER_CLIENT_STDOUT = "driverclient.stdout";
   private static final String EVALUATOR_STDERR = "evaluator.stderr";
   private static final String EVALUATOR_STDOUT = "evaluator.stdout";
   private static final String DRIVER_HTTP_ENDPOINT_FILE_NAME = "DriverHttpEndpoint.txt";
@@ -203,6 +205,20 @@ public final class REEFFileNames {
   }
 
   /**
+   * @return The name used within the current working directory of the driver client to redirect stderr to.
+   */
+  public String getDriverClientStderrFileName() {
+    return DRIVER_CLIENT_STDERR;
+  }
+
+  /**
+   * @return The name used within the current working directory of the driver client to redirect stdout to.
+   */
+  public String getDriverClientStdoutFileName() {
+    return DRIVER_CLIENT_STDOUT;
+  }
+
+  /**
    * @return The prefix used whenever REEF is asked to create an Evaluator folder, e.g. for staging.
    */
   public String getEvaluatorFolderPrefix() {
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/utils/DefaultExceptionCodec.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/utils/DefaultExceptionCodec.java
index 91b45ce..9081745 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/utils/DefaultExceptionCodec.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/utils/DefaultExceptionCodec.java
@@ -41,11 +41,13 @@ final class DefaultExceptionCodec implements ExceptionCodec {
   @Override
   public Optional<Throwable> fromBytes(final byte[] bytes) {
     try {
-      return Optional.<Throwable>of((Throwable) SerializationUtils.deserialize(bytes));
-    } catch (SerializationException | IllegalArgumentException e) {
-      LOG.log(Level.FINE, "Unable to deserialize a Throwable.", e);
-      return Optional.empty();
+      if (bytes != null && bytes.length > 0) {
+        return Optional.of((Throwable) SerializationUtils.deserialize(bytes));
+      }
+    } catch (final SerializationException | IllegalArgumentException e) {
+      LOG.log(Level.WARNING, "Unable to deserialize a Throwable.", e);
     }
+    return Optional.empty();
   }
 
   @Override
diff --git a/lang/java/reef-tests/pom.xml b/lang/java/reef-tests/pom.xml
index b096e39..d39734b 100644
--- a/lang/java/reef-tests/pom.xml
+++ b/lang/java/reef-tests/pom.xml
@@ -34,6 +34,7 @@ under the License.
 
     <properties>
         <rootPath>${basedir}/../../..</rootPath>
+        <protobuf.version>3.5.1</protobuf.version>
     </properties>
 
     <dependencies>
@@ -79,6 +80,16 @@ under the License.
         </dependency>
         <dependency>
             <groupId>${project.groupId}</groupId>
+            <artifactId>reef-bridge-proto-java</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.protobuf</groupId>
+            <artifactId>protobuf-java</artifactId>
+            <version>${protobuf.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
             <artifactId>vortex</artifactId>
             <version>${project.version}</version>
         </dependency>
diff --git a/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/driver/FailBridgeClient.java b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/driver/FailBridgeClient.java
new file mode 100644
index 0000000..2524dd9
--- /dev/null
+++ b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/driver/FailBridgeClient.java
@@ -0,0 +1,101 @@
+/*
+ * 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.tests.fail.driver;
+
+import org.apache.reef.annotations.audience.ClientSide;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.DriverClientConfiguration;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tests.TestDriverLauncher;
+import org.apache.reef.tests.fail.util.FailBridgeClientUtils;
+import org.apache.reef.util.EnvironmentUtils;
+import org.apache.reef.util.OSUtils;
+
+import java.io.IOException;
+
+/**
+ * fail bridge client.
+ */
+@Private
+@ClientSide
+public final class FailBridgeClient {
+
+  private static final Tang TANG = Tang.Factory.getTang();
+
+  private static Configuration buildDriverConfig(final Class<?> failMsgClass) {
+
+    final Configuration driverConfig = DriverClientConfiguration.CONF
+        .set(DriverClientConfiguration.ON_DRIVER_STARTED, FailDriver.StartHandler.class)
+        .set(DriverClientConfiguration.ON_DRIVER_STOP, FailDriver.StopHandler.class)
+        .set(DriverClientConfiguration.ON_EVALUATOR_ALLOCATED, FailDriver.AllocatedEvaluatorHandler.class)
+        .set(DriverClientConfiguration.ON_EVALUATOR_COMPLETED, FailDriver.CompletedEvaluatorHandler.class)
+        .set(DriverClientConfiguration.ON_EVALUATOR_FAILED, FailDriver.FailedEvaluatorHandler.class)
+        .set(DriverClientConfiguration.ON_CONTEXT_ACTIVE, FailDriver.ActiveContextHandler.class)
+        .set(DriverClientConfiguration.ON_CONTEXT_MESSAGE, FailDriver.ContextMessageHandler.class)
+        .set(DriverClientConfiguration.ON_CONTEXT_CLOSED, FailDriver.ClosedContextHandler.class)
+        .set(DriverClientConfiguration.ON_CONTEXT_FAILED, FailDriver.FailedContextHandler.class)
+        .set(DriverClientConfiguration.ON_TASK_RUNNING, FailDriver.RunningTaskHandler.class)
+        .set(DriverClientConfiguration.ON_TASK_SUSPENDED, FailDriver.SuspendedTaskHandler.class)
+        .set(DriverClientConfiguration.ON_TASK_MESSAGE, FailDriver.TaskMessageHandler.class)
+        .set(DriverClientConfiguration.ON_TASK_FAILED, FailDriver.FailedTaskHandler.class)
+        .set(DriverClientConfiguration.ON_TASK_COMPLETED, FailDriver.CompletedTaskHandler.class)
+        .build();
+
+    return TANG.newConfigurationBuilder(driverConfig)
+        .bindNamedParameter(FailDriver.FailMsgClassName.class, failMsgClass.getName())
+        .build();
+  }
+
+  public static LauncherStatus runClient(
+      final Class<?> failMsgClass,
+      final Configuration runtimeConfig,
+      final int timeOut) throws IOException, InjectionException {
+    ClientProtocol.DriverClientConfiguration.Builder builder =
+        ClientProtocol.DriverClientConfiguration.newBuilder()
+        .setJobid("Fail_" + failMsgClass.getSimpleName())
+        .addGlobalLibraries(EnvironmentUtils.getClassLocation(FailDriver.class));
+    builder.setOperatingSystem(
+        OSUtils.isWindows() ?
+            ClientProtocol.DriverClientConfiguration.OS.WINDOWS :
+            ClientProtocol.DriverClientConfiguration.OS.LINUX);
+
+    return runClient(failMsgClass, runtimeConfig, builder.build(), timeOut);
+  }
+
+  public static LauncherStatus runClient(
+      final Class<?> failMsgClass,
+      final Configuration runtimeConfig,
+      final ClientProtocol.DriverClientConfiguration driverClientConfiguration,
+      final int timeOut) throws InjectionException, IOException {
+    final Configuration driverServiceConfiguration =
+        FailBridgeClientUtils.setupDriverService(
+            runtimeConfig,
+            buildDriverConfig(failMsgClass),
+            driverClientConfiguration);
+    return TestDriverLauncher.getLauncher(runtimeConfig).run(driverServiceConfiguration, timeOut);
+  }
+
+  private FailBridgeClient() {
+  }
+}
diff --git a/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/task/BridgeClient.java b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/task/BridgeClient.java
new file mode 100644
index 0000000..5317a7b
--- /dev/null
+++ b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/task/BridgeClient.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.tests.fail.task;
+
+import org.apache.reef.annotations.audience.ClientSide;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.DriverClientConfiguration;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.task.Task;
+import org.apache.reef.tests.TestDriverLauncher;
+import org.apache.reef.tests.fail.util.FailBridgeClientUtils;
+import org.apache.reef.util.EnvironmentUtils;
+import org.apache.reef.util.OSUtils;
+
+import java.io.IOException;
+
+/**
+ * Fail task bridge client.
+ */
+@Private
+@ClientSide
+public final class BridgeClient  {
+
+  /**
+   * Empty private constructor to prohibit instantiation of utility class.
+   */
+  private BridgeClient() {
+  }
+
+  public static LauncherStatus run(
+      final Class<? extends Task> failTaskClass,
+      final Configuration runtimeConfig,
+      final int timeOut) throws IOException, InjectionException {
+    ClientProtocol.DriverClientConfiguration.Builder builder =
+        ClientProtocol.DriverClientConfiguration.newBuilder()
+            .setJobid("Fail_" + failTaskClass.getSimpleName())
+            .addGlobalLibraries(EnvironmentUtils.getClassLocation(Driver.class));
+    builder.setOperatingSystem(
+        OSUtils.isWindows() ?
+            ClientProtocol.DriverClientConfiguration.OS.WINDOWS :
+            ClientProtocol.DriverClientConfiguration.OS.LINUX);
+
+    return run(failTaskClass, runtimeConfig, builder.build(), timeOut);
+  }
+
+  public static LauncherStatus run(
+      final Class<? extends Task> failTaskClass,
+      final Configuration runtimeConfig,
+      final ClientProtocol.DriverClientConfiguration driverClientConfiguration,
+      final int timeOut) throws InjectionException, IOException {
+
+    final Configuration driverConfig = DriverClientConfiguration.CONF
+        .set(DriverClientConfiguration.ON_EVALUATOR_ALLOCATED, Driver.AllocatedEvaluatorHandler.class)
+        .set(DriverClientConfiguration.ON_TASK_RUNNING, Driver.RunningTaskHandler.class)
+        .set(DriverClientConfiguration.ON_CONTEXT_ACTIVE, Driver.ActiveContextHandler.class)
+        .set(DriverClientConfiguration.ON_DRIVER_STARTED, Driver.StartHandler.class)
+        .build();
+
+    final JavaConfigurationBuilder cb = Tang.Factory.getTang().newConfigurationBuilder();
+    cb.addConfiguration(driverConfig);
+    cb.bindNamedParameter(Driver.FailTaskName.class, failTaskClass.getSimpleName());
+
+    final Configuration driverServiceConfiguration =
+        FailBridgeClientUtils.setupDriverService(
+            runtimeConfig,
+            cb.build(),
+            driverClientConfiguration);
+    return TestDriverLauncher.getLauncher(runtimeConfig).run(driverServiceConfiguration, timeOut);
+  }
+}
diff --git a/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/util/FailBridgeClientUtils.java b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/util/FailBridgeClientUtils.java
new file mode 100644
index 0000000..b79f619
--- /dev/null
+++ b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/util/FailBridgeClientUtils.java
@@ -0,0 +1,102 @@
+/*
+ * 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.tests.fail.util;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.annotations.audience.ClientSide;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.bridge.driver.client.JavaDriverClientLauncher;
+import org.apache.reef.bridge.driver.service.DriverServiceConfigurationProvider;
+import org.apache.reef.bridge.driver.service.grpc.GRPCDriverServiceConfigurationProvider;
+import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.runtime.common.files.ClasspathProvider;
+import org.apache.reef.runtime.common.files.REEFFileNames;
+import org.apache.reef.runtime.common.launch.JavaLaunchCommandBuilder;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Fail bridge client utilities.
+ */
+@Private
+@ClientSide
+public final class FailBridgeClientUtils {
+
+  private static final Tang TANG = Tang.Factory.getTang();
+
+  /**
+   * Setup the bridge service configuration.
+   * @param runtimeConfiguration runtime configuration
+   * @param driverClientConfiguration driver client configuration
+   * @param driverClientConfigurationProto protocol arguments
+   * @return bridge service configuration
+   * @throws IOException
+   * @throws InjectionException
+   */
+  public static Configuration setupDriverService(
+      final Configuration runtimeConfiguration,
+      final Configuration driverClientConfiguration,
+      final ClientProtocol.DriverClientConfiguration driverClientConfigurationProto)
+      throws IOException, InjectionException {
+    final File driverClientConfigurationFile = File.createTempFile("driverclient", ".conf");
+    // Write driver client configuration to a file
+    final Injector driverClientInjector = TANG.newInjector(driverClientConfiguration);
+    final ConfigurationSerializer configurationSerializer =
+        driverClientInjector.getInstance(ConfigurationSerializer.class);
+    configurationSerializer.toFile(driverClientConfiguration, driverClientConfigurationFile);
+
+    final Injector runtimeInjector = TANG.newInjector(runtimeConfiguration);
+    final REEFFileNames fileNames = runtimeInjector.getInstance(REEFFileNames.class);
+    final ClasspathProvider classpathProvider = runtimeInjector.getInstance(ClasspathProvider.class);
+    final List<String> launchCommand = new JavaLaunchCommandBuilder(JavaDriverClientLauncher.class, null)
+        .setConfigurationFilePaths(
+            Collections.singletonList("./" + fileNames.getLocalFolderPath() + "/" +
+                driverClientConfigurationFile.getName()))
+        .setJavaPath("java")
+        .setClassPath(StringUtils.join(classpathProvider.getDriverClasspath(),
+            driverClientConfigurationProto.getOperatingSystem() ==
+                ClientProtocol.DriverClientConfiguration.OS.WINDOWS ? ";" : ":"))
+        .build();
+    final String cmd = StringUtils.join(launchCommand, ' ');
+    final ClientProtocol.DriverClientConfiguration driverServiceConfiguration =
+        ClientProtocol.DriverClientConfiguration.newBuilder(driverClientConfigurationProto)
+            .setDriverClientLaunchCommand(cmd)
+            .addLocalFiles(driverClientConfigurationFile.getAbsolutePath())
+            .build();
+    final DriverServiceConfigurationProvider driverServiceConfigurationProvider = TANG.newInjector(
+        TANG.newConfigurationBuilder()
+            .bindImplementation(DriverServiceConfigurationProvider.class,
+                GRPCDriverServiceConfigurationProvider.class)
+            .build())
+        .getInstance(DriverServiceConfigurationProvider.class);
+    return driverServiceConfigurationProvider.getDriverServiceConfiguration(driverServiceConfiguration);
+  }
+
+  private FailBridgeClientUtils() {
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/util/package-info.java
similarity index 69%
copy from lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
copy to lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/util/package-info.java
index 9967d9b..d271af2 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-tests/src/main/java/org/apache/reef/tests/fail/util/package-info.java
@@ -16,20 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.reef.tests.fail;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
 /**
- * Test suite of tests covering failure scenarios.
+ * Utilities for Driver-side failures.
  */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-//    FailTaskTest.class,
-    FailDriverTest.class,
-    FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
-    })
-public final class FailTestSuite {
-}
+package org.apache.reef.tests.fail.util;
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailBridgeDriverTest.java b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailBridgeDriverTest.java
new file mode 100644
index 0000000..217811b
--- /dev/null
+++ b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailBridgeDriverTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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.tests.fail;
+
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.evaluator.AllocatedEvaluator;
+import org.apache.reef.driver.evaluator.CompletedEvaluator;
+import org.apache.reef.driver.task.CompletedTask;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.driver.task.SuspendedTask;
+import org.apache.reef.driver.task.TaskMessage;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.exceptions.BindException;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tests.TestEnvironment;
+import org.apache.reef.tests.TestEnvironmentFactory;
+import org.apache.reef.tests.TestUtils;
+import org.apache.reef.tests.fail.driver.FailBridgeClient;
+import org.apache.reef.tests.fail.driver.FailClient;
+import org.apache.reef.tests.fail.driver.FailDriver;
+import org.apache.reef.tests.library.exceptions.SimulatedDriverFailure;
+import org.apache.reef.util.OSUtils;
+import org.apache.reef.wake.time.event.Alarm;
+import org.apache.reef.wake.time.event.StartTime;
+import org.apache.reef.wake.time.event.StopTime;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Run FailDriver with different types of failures.
+ */
+public class FailBridgeDriverTest {
+
+  private final TestEnvironment testEnvironment = TestEnvironmentFactory.getNewTestEnvironment();
+
+  @Before
+  public void setUp() throws Exception {
+    testEnvironment.setUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    this.testEnvironment.tearDown();
+  }
+
+  private void failOn(final Class<?> clazz) throws BindException, InjectionException {
+    try {
+      if (OSUtils.isLinux()) {
+        TestUtils.assertLauncherFailure(
+            FailBridgeClient.runClient(clazz,
+                this.testEnvironment.getRuntimeConfiguration(), this.testEnvironment.getTestTimeout()),
+            SimulatedDriverFailure.class);
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testFailDriverConstructor() throws BindException, InjectionException {
+    failOn(FailDriver.class);
+  }
+
+  @Test
+  public void testFailDriverStart() throws BindException, InjectionException {
+    failOn(StartTime.class);
+  }
+
+  @Test
+  public void testFailDriverAllocatedEvaluator() throws BindException, InjectionException {
+    failOn(AllocatedEvaluator.class);
+  }
+
+  @Test
+  public void testFailDriverActiveContext() throws BindException, InjectionException {
+    failOn(ActiveContext.class);
+  }
+
+  @Test
+  public void testFailDriverRunningTask() throws BindException, InjectionException {
+    failOn(RunningTask.class);
+  }
+
+  @Test
+  public void testFailDriverTaskMessage() throws BindException, InjectionException {
+    failOn(TaskMessage.class);
+  }
+
+  @Test
+  public void testFailDriverSuspendedTask() throws BindException, InjectionException {
+    failOn(SuspendedTask.class);
+  }
+
+  @Test
+  public void testFailDriverCompletedTask() throws BindException, InjectionException {
+    failOn(CompletedTask.class);
+  }
+
+  @Test
+  public void testFailDriverCompletedEvaluator() throws BindException, InjectionException {
+    failOn(CompletedEvaluator.class);
+  }
+
+  @Test
+  public void testFailDriverAlarm() throws BindException, InjectionException {
+    failOn(Alarm.class);
+  }
+
+  @Test
+  public void testFailDriverStop() throws BindException, InjectionException {
+    failOn(StopTime.class);
+  }
+
+  @Test
+  public void testDriverCompleted() throws BindException, InjectionException {
+    final Configuration runtimeConfiguration = this.testEnvironment.getRuntimeConfiguration();
+    // FailDriverTest can be replaced with any other class never used in FailDriver
+    final LauncherStatus status = FailClient.runClient(
+        FailDriverTest.class, runtimeConfiguration, this.testEnvironment.getTestTimeout());
+    Assert.assertEquals(LauncherStatus.COMPLETED, status);
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailBridgeTaskTest.java b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailBridgeTaskTest.java
new file mode 100644
index 0000000..c9e3de5
--- /dev/null
+++ b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailBridgeTaskTest.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.tests.fail;
+
+import org.apache.reef.tang.exceptions.BindException;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.task.Task;
+import org.apache.reef.tests.TestEnvironment;
+import org.apache.reef.tests.TestEnvironmentFactory;
+import org.apache.reef.tests.TestUtils;
+import org.apache.reef.tests.fail.task.*;
+import org.apache.reef.tests.library.exceptions.SimulatedTaskFailure;
+import org.apache.reef.util.OSUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Run Driver with different types of failures in the Task.
+ */
+public final class FailBridgeTaskTest {
+
+  private final TestEnvironment testEnvironment = TestEnvironmentFactory.getNewTestEnvironment();
+
+  @Before
+  public void setUp() throws Exception {
+    testEnvironment.setUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    this.testEnvironment.tearDown();
+  }
+
+  private void failOn(
+      final Class<? extends Task> failTaskClass) throws BindException, InjectionException, IOException {
+    if (OSUtils.isLinux()) {
+      TestUtils.assertLauncherFailure(
+          BridgeClient.run(failTaskClass,
+              this.testEnvironment.getRuntimeConfiguration(),
+              this.testEnvironment.getTestTimeout()),
+          SimulatedTaskFailure.class);
+    }
+  }
+
+  @Test
+  public void testFailTask() throws BindException, InjectionException, IOException {
+    failOn(FailTask.class);
+  }
+
+  @Test
+  public void testFailTaskCall() throws BindException, InjectionException, IOException {
+    failOn(FailTaskCall.class);
+  }
+
+  @Test
+  public void testFailTaskMsg() throws BindException, InjectionException, IOException {
+    failOn(FailTaskMsg.class);
+  }
+
+  @Test
+  public void testFailTaskSuspend() throws BindException, InjectionException, IOException {
+    failOn(FailTaskSuspend.class);
+  }
+
+  @Test
+  public void testFailTaskStart() throws BindException, InjectionException, IOException {
+    failOn(FailTaskStart.class);
+  }
+
+  @Test
+  public void testFailTaskStop() throws BindException, InjectionException, IOException {
+    failOn(FailTaskStop.class);
+  }
+
+  @Test
+  public void testFailTaskClose() throws BindException, InjectionException, IOException {
+    failOn(FailTaskClose.class);
+  }
+}
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
index 9967d9b..a3481e4 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
+++ b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/fail/FailTestSuite.java
@@ -29,7 +29,9 @@ import org.junit.runners.Suite;
 //    FailTaskTest.class,
     FailDriverTest.class,
     FailDriverDelayedMsgTest.class,
-    DriverFailOnFailTest.class
+    DriverFailOnFailTest.class,
+    FailBridgeDriverTest.class,
+    FailBridgeTaskTest.class
     })
 public final class FailTestSuite {
 }
diff --git a/pom.xml b/pom.xml
index 8d58a93..7055ed8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -50,7 +50,7 @@ under the License.
         <reef.log.dir>${project.build.directory}/log</reef.log.dir>
         <bundle.snappy>false</bundle.snappy>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-        <hadoop.version>2.7.0</hadoop.version>
+        <hadoop.version>2.7.5</hadoop.version>
         <spark.version>2.1.0</spark.version>
         <avro.version>1.8.1</avro.version>
         <parquet.version>1.9.0</parquet.version>
@@ -66,6 +66,17 @@ under the License.
         <kryo.version>3.0.3</kryo.version>
         <kryo-serializers.version>0.37</kryo-serializers.version>
         <fast-classpath-scanner.version>2.4.5</fast-classpath-scanner.version>
+        <maven.assembly>3.1.0</maven.assembly>
+        <grpc.version>1.12.0</grpc.version><!-- CURRENT_GRPC_VERSION -->
+        <guava.version>20.0</guava.version>
+        <build-helper-maven-plugin.version>1.9.1</build-helper-maven-plugin.version>
+        <maven-antrun-plugin.version>1.8</maven-antrun-plugin.version>
+        <maven-dependency-plugin.version>2.10</maven-dependency-plugin.version>
+        <maven-shade-plugin.version>2.4.2</maven-shade-plugin.version>
+        <os-maven-plugin.version>1.4.1.Final</os-maven-plugin.version>
+        <!-- do not move beyond this version since it adds an incompatible guava dependency -->
+        <xolstice.version>0.5.0</xolstice.version>
+        <commons-codec.version>1.9</commons-codec.version>
         <rootPath>${user.dir}</rootPath>
     </properties>
 
@@ -289,6 +300,8 @@ under the License.
                             <exclude>REEF_STANDALONE_RUNTIME/**</exclude>
                             <!-- Error logs -->
                             <exclude>**/*.log</exclude>
+                            <!-- Shading configuration -->
+                            <exclude>**/dependency-reduced-pom.xml</exclude>
                             <!-- The Visual Studio and Nuget build files -->
                             <exclude>**/.vs/**</exclude>
                             <exclude>**/*.sln*</exclude>
@@ -298,8 +311,10 @@ under the License.
                             <exclude>**/*.sdf*</exclude>
                             <exclude>**/*.snk</exclude>
                             <exclude>**/*.opendb</exclude>
+                            <exclude>**/*.resx</exclude>
                             <!-- The below are auto generated during the .Net build -->
                             <exclude>**/bin/**</exclude>
+                            <exclude>**/generated/**</exclude>
                             <exclude>**/obj/**</exclude>
                             <exclude>**/Release/**</exclude>
                             <exclude>**/Debug/**</exclude>
@@ -699,6 +714,16 @@ under the License.
             </dependency>
             <!-- End of SLF4J -->
 
+            <dependency>
+                <groupId>io.grpc</groupId>
+                <artifactId>grpc-all</artifactId>
+                <version>${grpc.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.google.guava</groupId>
+                <artifactId>guava</artifactId>
+                <version>${guava.version}</version>
+            </dependency>
 
             <dependency>
                 <groupId>io.netty</groupId>
@@ -707,6 +732,12 @@ under the License.
             </dependency>
 
             <dependency>
+                <groupId>commons-codec</groupId>
+                <artifactId>commons-codec</artifactId>
+                <version>${commons-codec.version}</version>
+            </dependency>
+
+            <dependency>
                 <groupId>cglib</groupId>
                 <artifactId>cglib</artifactId>
                 <version>3.1</version>
@@ -769,6 +800,7 @@ under the License.
         <module>lang/java/reef-applications</module>
         <module>lang/java/reef-bridge-client</module>
         <module>lang/java/reef-bridge-java</module>
+        <module>lang/java/reef-bridge-proto-java</module>
         <module>lang/java/reef-checkpoint</module>
         <module>lang/java/reef-common</module>
         <module>lang/java/reef-examples</module>