You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by tr...@apache.org on 2018/10/12 17:36:46 UTC

[reef] branch master updated: [REEF-2028] A new module containing the new .Net bridge (#1477)

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

truegrit 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 17b344c  [REEF-2028] A new module containing the new .Net bridge (#1477)
17b344c is described below

commit 17b344c2935ac0ab259365d2e51409fc65789d41
Author: Tyson Condie <tc...@users.noreply.github.com>
AuthorDate: Fri Oct 12 10:36:41 2018 -0700

    [REEF-2028] A new module containing the new .Net bridge (#1477)
    
    * [REEF-2025] A new module containing the new Java bridge
    
        This Jira 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.
    
    Pull Request:
        Closes #1477
---
 .gitignore                                         |   3 +-
 lang/common/proto/bridge/ClientProtocol.proto      |  90 ++-
 .../common/proto/bridge/DriverClientProtocol.proto |   6 +-
 .../common/proto/bridge/DriverCommonProtocol.proto |   2 +-
 .../proto/bridge/DriverServiceProtocol.proto       |  18 +-
 lang/cs/App.config                                 |   6 +-
 .../Client/Config/ClientConfiguration.cs           |  51 ++
 .../Client/Config/ClientParameters.cs              |  65 ++
 .../Config/DriverApplicationConfiguration.cs       | 219 ++++++
 .../Client/Config/DriverApplicationParameters.cs   | 164 +++++
 .../Client/Config/DriverRuntimeConfiguration.cs    | 148 +++++
 .../Client/Config/DriverRuntimeParameters.cs       |  89 +++
 .../Client/Config/OsType.cs}                       |  34 +-
 .../Runtime/AzureBatchRuntimeConfiguration.cs      |  85 +++
 .../Config/Runtime/AzureBatchRuntimeParameters.cs  |  58 ++
 .../Runtime/HdInsightRuntimeConfiguration.cs       |  82 +++
 .../Config/Runtime/HdInsightRuntimeParameters.cs   |  58 ++
 .../Config/Runtime/LocalRuntimeConfiguration.cs    |  51 ++
 .../Config/Runtime/LocalRuntimeParameters.cs}      |  17 +-
 .../Proto/AzureBatchRuntimeProtoProvider.cs        |  53 ++
 .../Runtime/Proto/HdInsightRuntimeProtoProvider.cs |  53 ++
 .../Config/Runtime/Proto/IRuntimeProtoProvider.cs} |  10 +-
 .../Runtime/Proto/LocalRuntimeProtoProvider.cs     |  49 ++
 .../Runtime/Proto/YarnRuntimeProtoProvider.cs      |  57 ++
 .../Client/Config/Runtime/Utils.cs                 |  47 ++
 .../Config/Runtime/YarnRuntimeConfiguration.cs     |  49 ++
 .../Client/Config/Runtime/YarnRuntimeParameters.cs |  57 ++
 .../Client/Default/DefaultCompletedJobHandler.cs}  |  27 +-
 .../Client/Default/DefaultFailedJobHandler.cs}     |  27 +-
 .../Client/Default/DefaultFailedRuntimeHandler.cs} |  27 +-
 .../Client/Default/DefaultJobMessageHandler.cs}    |  39 +-
 .../Client/Default/DefaultRunningJobHandler.cs}    |  27 +-
 .../Client/Default/DefaultSubmittedJobHandler.cs}  |  27 +-
 .../Client/Default/DefaultWakeErrorHandler.cs}     |  27 +-
 .../Client/Events/CompletedJob.cs}                 |  15 +-
 .../Client/Events/FailedJob.cs                     |  64 ++
 .../Client/Events/FailedRuntime.cs                 |  64 ++
 .../Client/Events/JobMessage.cs}                   |  17 +-
 .../Client/Events/RunningJob.cs}                   |  33 +-
 .../Client/Events/SubmittedJob.cs}                 |  15 +-
 .../Client/Events/WakeError.cs                     |  68 ++
 .../Client/IClientLauncher.cs}                     |  21 +-
 .../Client/IClientService.cs}                      |  24 +-
 .../Client}/JavaClientLauncher.cs                  | 126 ++--
 .../Driver/BridgeClock.cs                          | 119 ++++
 .../Driver/DispatchEventHandler.cs                 |  76 +++
 .../Driver/DriverBridge.cs                         | 353 ++++++++++
 .../Driver/DriverBridgeConfiguration.cs}           |  18 +-
 .../Driver/DriverBridgeEvaluatorRequestor.cs}      |  33 +-
 .../Driver/Events/BridgeActiveContext.cs           |  74 +++
 .../Driver/Events/BridgeAllocatedEvaluator.cs      | 163 +++++
 .../Driver/Events/BridgeClosedContext.cs           |  46 ++
 .../Driver/Events/BridgeCompletedEvaluator.cs}     |  15 +-
 .../Driver/Events/BridgeCompletedTask.cs}          |  20 +-
 .../Driver/Events/BridgeContextMessage.cs}         |  19 +-
 .../Driver/Events/BridgeDriverRestartCompleted.cs} |  18 +-
 .../Driver/Events/BridgeDriverRestarted.cs}        |  21 +-
 .../Driver/Events/BridgeDriverStarted.cs}          |  16 +-
 .../Driver/Events/BridgeDriverStopped.cs}          |  16 +-
 .../Driver/Events/BridgeFailedContext.cs           |  45 ++
 .../Driver/Events/BridgeFailedEvaluator.cs}        |  45 +-
 .../Driver/Events/BridgeFailedTask.cs              | 130 ++++
 .../Driver/Events/BridgeRunningTask.cs             |  64 ++
 .../Driver/Events/BridgeSuspendedTask.cs}          |  20 +-
 .../Driver/Events/BridgeTaskMessage.cs}            |  19 +-
 .../Driver/Events/EvaluatorDescriptor.cs           |  57 ++
 .../Driver/Events/NodeDescriptor.cs                |  58 ++
 .../Driver/IDriverClientService.cs}                |  22 +-
 .../Driver/IDriverServiceClient.cs                 | 128 ++++
 .../Org.Apache.REEF.Bridge.Core.Common.csproj      |  36 +
 .../Properties/AssemblyInfo.cs                     |   5 +-
 .../DriverLauncher.cs                              |  90 +++
 .../Org.Apache.REEF.Bridge.Core.Driver.csproj      |  34 +
 .../Client/ClientLauncher.cs                       | 235 +++++++
 .../Client/ClientLauncherFactory.cs                |  65 ++
 .../Client/ClientService.cs                        | 216 ++++++
 .../Client/DriverClientParameters.cs               |  78 +++
 .../Driver/DriverClientService.cs                  | 738 +++++++++++++++++++++
 .../Driver/DriverServiceClient.cs                  | 261 ++++++++
 .../Driver/DriverServicePort.cs}                   |  13 +-
 .../Driver/GRPCUtils.cs                            |  54 ++
 .../Org.Apache.REEF.Bridge.Core.Grpc.csproj        |  44 ++
 .../FailedTask.cs}                                 |  28 +-
 .../HelloDriver.cs                                 | 133 ++++
 .../HelloREEF.cs                                   | 148 +++++
 .../HelloTask.cs}                                  |  27 +-
 .../Org.Apache.REEF.Bridge.Core.HelloREEF.csproj   |  45 ++
 .../Run.cs}                                        |  17 +-
 .../Org.Apache.REEF.Bridge.Core.Proto/Grpc.targets | 119 ++++
 .../Org.Apache.REEF.Bridge.Core.Proto.csproj       |  85 +++
 .../Fail/Driver/FailDriver.cs                      | 406 ++++++++++++
 .../Fail/Driver/FailDriverClient.cs                |  94 +++
 .../Fail/Driver/NoopTask.cs                        | 151 +++++
 .../Fail/Exception/DriverSideFailure.cs}           |  22 +-
 .../Fail/Exception/SimulatedDriverFailure.cs}      |  22 +-
 .../Fail/Exception/SimulatedTaskFailure.cs         |  37 ++
 .../Fail/Task/BlockingCallTaskBase.cs              |  70 ++
 .../Fail/Task/Driver.cs                            | 185 ++++++
 .../Fail/Task/FailTask.cs}                         |  36 +-
 .../Fail/Task/FailTaskCall.cs}                     |  35 +-
 .../Fail/Task/FailTaskClient.cs                    |  82 +++
 .../Fail/Task/FailTaskClose.cs}                    |  38 +-
 .../Fail/Task/FailTaskMsg.cs}                      |  38 +-
 .../Fail/Task/FailTaskStart.cs}                    |  38 +-
 .../Fail/Task/FailTaskStop.cs}                     |  37 +-
 .../Fail/Task/FailTaskSuspend.cs}                  |  38 +-
 .../Fail/Test/FailDriverTest.cs                    | 116 ++++
 .../Fail/Test/FailTaskTest.cs                      |  77 +++
 .../Fail/Test/TestUtils.cs                         |  49 ++
 .../Org.Apache.REEF.Bridge.Core.Tests.csproj       |  33 +
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Org.Apache.REEF.Bridge.JAR.csproj              |  14 +-
 .../cs/Org.Apache.REEF.Client/API/JobParameters.cs |  11 +
 .../API/JobParametersBuilder.cs                    |  13 +
 .../Common/JavaClientLauncher.cs                   |  11 +-
 .../Org.Apache.REEF.Client.csproj                  |   4 +-
 .../Properties/AssemblyInfo.cs                     |   5 +
 .../Client/ICompletedJob.cs}                       |  11 +-
 .../Client/IFailedJob.cs}                          |   9 +-
 .../Client/IFailedRuntime.cs}                      |  11 +-
 .../Client/IJobMessage.cs}                         |  11 +-
 .../Client/IRunningJob.cs}                         |  20 +-
 .../Client/ISubmittedJob.cs}                       |  14 +-
 .../Client/IWakeError.cs}                          |   9 +-
 .../Client/LauncherStatus.cs                       | 109 +++
 .../Exceptions/NonSerializableJobException.cs      |  69 ++
 .../Properties/AssemblyInfo.cs                     |   4 +
 .../Runtime/Evaluator/Task/TaskRuntime.cs          |  54 +-
 .../Defaults/DefaultTaskFailureHandler.cs          |  16 +-
 .../IDriverStopped.cs}                             |  12 +-
 .../Properties/AssemblyInfo.cs                     |   3 +-
 lang/cs/Org.Apache.REEF.Evaluator/Evaluator.cs     |   4 +-
 .../TestAzureBlobFileSystemE2E.cs                  |   3 +-
 .../Functional/Bridge/TestCloseTask.cs             |   2 +-
 .../Functional/Bridge/TestDisposeTasks.cs          | 297 ---------
 .../User/ReceiveTaskMessageExceptionTest.cs        |  12 +-
 .../Failure/User/TaskCloseExceptionTest.cs         |  58 +-
 .../Failure/User/TaskSuspendExceptionTest.cs       | 159 -----
 .../Functional/FaultTolerant/TestResubmitTask.cs   |  51 +-
 .../Remote/Parameters/TcpPortRangeStart.cs         |   2 +-
 lang/cs/Org.Apache.REEF.sln                        | 118 ++++
 lang/cs/Org.Apache.REEF.sln.DotSettings            |   6 +
 lang/cs/pomversion.targets                         |  18 +-
 .../reef/bridge/client/grpc/ClientLauncher.java    | 165 +++++
 .../reef/bridge/client/grpc/ClientService.java     | 323 +++++++++
 .../grpc/ClientServiceConfiguration.java}          |  29 +-
 .../JavaClientLauncher.java}                       |  93 +--
 .../reef/bridge/client/grpc/package-info.java}     |  20 +-
 .../client/grpc/parameters/ClientServerPort.java}  |  24 +-
 .../client/grpc/parameters/package-info.java}      |  20 +-
 .../driver/client/DriverClientConfiguration.java   |   1 -
 .../reef/bridge/driver/common/grpc/GRPCUtils.java  |   5 +-
 ...cher.java => RuntimeConfigurationProvider.java} |  12 +-
 ...r.java => AzureBatchConfigurationProvider.java} |  24 +-
 .../hdinsight/HDInsightConfigurationProvider.java  |  79 +++
 .../driver/launch/hdinsight/package-info.java}     |  20 +-
 ...uncher.java => LocalConfigurationProvider.java} |  25 +-
 ...auncher.java => YarnConfigurationProvider.java} |  36 +-
 .../reef/bridge/driver/service/DriverService.java  |   3 +-
 .../driver/service/DriverServiceConfiguration.java |  10 +-
 .../driver/service/grpc/GRPCDriverService.java     | 713 +++++++++++---------
 .../GRPCDriverServiceConfigurationProvider.java    |   1 +
 .../reef/bridge/examples/hello/HelloREEF.java      |   4 +-
 .../reef/driver/evaluator/DotNetProcess.java       |  80 +++
 ...valuatorType.java => DotNetProcessFactory.java} |  33 +-
 .../reef/driver/evaluator/EvaluatorType.java       |   4 +
 .../reef/runtime/common/client/RunningJobImpl.java |  10 +-
 .../driver/evaluator/AllocatedEvaluatorImpl.java   |   7 +-
 .../common/evaluator/EvaluatorConfiguration.java   |   1 +
 .../common/launch/DotNetLaunchCommandBuilder.java  |  93 +++
 .../reef/tests/fail/driver/FailBridgeClient.java   |   2 +-
 .../apache/reef/tests/fail/task/BridgeClient.java  |   2 +-
 .../apache/reef/tests/LocalTestEnvironment.java    |   2 +-
 .../reef/tests/fail/FailBridgeDriverTest.java      |   2 +-
 pom.xml                                            |   6 +-
 175 files changed, 9229 insertions(+), 1705 deletions(-)

diff --git a/.gitignore b/.gitignore
index 1c6af07..f7272db 100644
--- a/.gitignore
+++ b/.gitignore
@@ -8,10 +8,9 @@ REEF_MESOS_RUNTIME
 # Files generated during builds
 # ----------------------------------------------------------------------
 target
-generated
+generated-sources
 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
index 3013b2f..b66af19 100644
--- a/lang/common/proto/bridge/ClientProtocol.proto
+++ b/lang/common/proto/bridge/ClientProtocol.proto
@@ -21,10 +21,77 @@ syntax = "proto3";
 
 option java_package = "org.apache.reef.bridge.proto";
 option java_outer_classname = "ClientProtocol";
-option csharp_namespace = "Org.Apache.REEF.Bridge.Proto";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Core.Proto";
 
 package driverbridge;
 
+import "DriverCommonProtocol.proto";
+
+// Bridge Client RPC and messages
+service BridgeClient {
+
+  rpc RegisterREEFClient (REEFClientRegistration) returns (Void) {}
+
+  rpc JobMessageHandler (JobMessageEvent) returns (Void) {}
+
+  rpc JobSumittedHandler (JobSubmittedEvent) returns (Void) {}
+
+  rpc JobRunningHandler (JobRunningEvent) returns (Void) {}
+
+  rpc JobCompletedHandler (JobCompletedEvent) returns (Void) {}
+
+  rpc JobFailedHandler (JobFailedEvent) returns (Void) {}
+
+  rpc RuntimeErrorHandler (ExceptionInfo) returns (Void) {}
+
+  rpc WakeErrorHandler (ExceptionInfo) returns (Void) {}
+}
+
+message REEFClientRegistration {
+  string hostname = 1;
+  uint32 port = 2;
+}
+
+message JobMessageEvent {
+  string job_id = 1;
+  bytes message = 2;
+}
+
+message JobSubmittedEvent {
+  string job_id = 1;
+}
+
+message JobRunningEvent {
+  string job_id = 1;
+}
+
+message JobCompletedEvent {
+  string job_id = 1;
+}
+
+message JobFailedEvent {
+  string job_id = 1;
+  ExceptionInfo exception = 2;
+}
+
+// REEF Client RPC and messages
+service REEFClient {
+
+  rpc DriverControlHandler (DriverControlOp) returns (Void) {}
+}
+
+message DriverControlOp {
+  string job_id = 1;
+
+  bytes message = 2;
+
+  enum Operation {
+    CLOSE = 0;
+    MESSAGE = 1;
+  }
+  Operation operation = 3;
+}
+
 /*
  * Local runtime parameters.
  */
@@ -70,9 +137,24 @@ message AzureBatchRuntimeParameters {
 }
 
 /*
- * Mesos runtime parameters.
+ * HD Insight runtime parameters.
  */
-message MesosRuntimeParameters {
+message HDIRuntimeParameters {
+  bool unsafe = 1;
+
+  // HD Insight Account Information
+  string hdi_user_name = 2;
+
+  string hdi_password = 3;
+
+  string hdi_url = 4;
+
+  // Azure Storage Account Information
+  string azure_storage_account_name = 5;
+
+  string azure_storage_account_key = 6;
+
+  string azure_storage_container_name = 7;
 }
 
 /*
@@ -92,7 +174,7 @@ message DriverClientConfiguration {
     LocalRuntimeParameters local_runtime = 5;
     YarnRuntimeParameters yarn_runtime = 6;
     AzureBatchRuntimeParameters azbatch_runtime = 7;
-    MesosRuntimeParameters mesos_runtime = 8;
+    HDIRuntimeParameters hdi_runtime = 8;
   }
 
   // The command to launch the driver client
diff --git a/lang/common/proto/bridge/DriverClientProtocol.proto b/lang/common/proto/bridge/DriverClientProtocol.proto
index 7411e8d..2f29c09 100644
--- a/lang/common/proto/bridge/DriverClientProtocol.proto
+++ b/lang/common/proto/bridge/DriverClientProtocol.proto
@@ -23,7 +23,7 @@ syntax = "proto3";
 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";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Core.Proto";
 
 package driverbridge;
 
@@ -121,6 +121,7 @@ message StopTimeInfo {
 // Information associated with an alarm that was set.
 message AlarmTriggerInfo {
   string alarm_id = 1;
+  int64 timestamp = 2;
 }
 
 // Evaluator descriptor information.
@@ -156,6 +157,9 @@ message EvaluatorInfo {
      string message = 1;
      repeated string failed_contexts = 2;
      string failed_task_id = 3;
+
+     // Exception specific to failed evaluator
+     ExceptionInfo exception = 5;
   }
   FailureInfo failure = 2;
 
diff --git a/lang/common/proto/bridge/DriverCommonProtocol.proto b/lang/common/proto/bridge/DriverCommonProtocol.proto
index 439b203..cc8e17c 100644
--- a/lang/common/proto/bridge/DriverCommonProtocol.proto
+++ b/lang/common/proto/bridge/DriverCommonProtocol.proto
@@ -22,7 +22,7 @@ syntax = "proto3";
 // 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";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Core.Proto";
 
 package driverbridge;
 
diff --git a/lang/common/proto/bridge/DriverServiceProtocol.proto b/lang/common/proto/bridge/DriverServiceProtocol.proto
index d9f8ffa..7f65563 100644
--- a/lang/common/proto/bridge/DriverServiceProtocol.proto
+++ b/lang/common/proto/bridge/DriverServiceProtocol.proto
@@ -23,7 +23,7 @@ syntax = "proto3";
 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";
+option csharp_namespace = "Org.Apache.REEF.Bridge.Core.Proto";
 
 package driverbridge;
 
@@ -82,6 +82,8 @@ message ResourceRequest {
   bool relax_locality = 7;
 
   string runtime_name = 8;
+
+  string node_label = 9;
 }
 
 // Request for an alarm to be set
@@ -115,8 +117,11 @@ message AllocatedEvaluatorRequest {
   // Context configuration
   string context_configuration = 6;
 
+  // Service configuration
+  string service_configuration = 7;
+
   // Task configuration
-  string task_configuration = 7;
+  string task_configuration = 8;
 
   message EvaluatorProcessRequest {
     int32 memory_mb = 1;
@@ -128,8 +133,15 @@ message AllocatedEvaluatorRequest {
     string standard_err = 4;
 
     repeated string options = 5;
+
+    enum Type {
+      JVM = 0;
+      CLR = 1;
+      DOTNET = 2;
+    }
+    Type process_type = 10;
   }
-  EvaluatorProcessRequest set_process = 8;
+  EvaluatorProcessRequest set_process = 10;
 }
 
 // Active context request.
diff --git a/lang/cs/App.config b/lang/cs/App.config
index 51d8c4f..fde648e 100644
--- a/lang/cs/App.config
+++ b/lang/cs/App.config
@@ -38,7 +38,11 @@ under the License.
       <dependentAssembly>
         <assemblyIdentity name="Microsoft.Data.Edm" publicKeyToken="31bf3856ad364e35" culture="neutral" />
         <bindingRedirect oldVersion="0.0.0.0-5.8.1.0" newVersion="5.8.1.0" />
+	  </dependentAssembly>
+	  <dependentAssembly>
+        <assemblyIdentity name="System.Interactive.Async" publicKeyToken="94bc3704cddfc263" culture="neutral" />
+        <bindingRedirect oldVersion="0.0.0.0-3.0.3000.0" newVersion="3.0.3000.0" />
       </dependentAssembly>
     </assemblyBinding>
   </runtime>
-</configuration>
+</configuration>
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/ClientConfiguration.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/ClientConfiguration.cs
new file mode 100644
index 0000000..1a498e8
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/ClientConfiguration.cs
@@ -0,0 +1,51 @@
+// 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.
+
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+using System;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config
+{
+    public sealed class ClientConfiguration : ConfigurationModuleBuilder
+    {
+        public static readonly OptionalParameter<IObserver<IRunningJob>> OnRunningJob = new OptionalParameter<IObserver<IRunningJob>>();
+
+        public static readonly OptionalParameter<IObserver<ISubmittedJob>> OnSubmittedJob = new OptionalParameter<IObserver<ISubmittedJob>>();
+
+        public static readonly OptionalParameter<IObserver<ICompletedJob>> OnCompletedJob = new OptionalParameter<IObserver<ICompletedJob>>();
+
+        public static readonly OptionalParameter<IObserver<IFailedJob>> OnFailedJob = new OptionalParameter<IObserver<IFailedJob>>();
+
+        public static readonly OptionalParameter<IObserver<IJobMessage>> OnJobMessage = new OptionalParameter<IObserver<IJobMessage>>();
+
+        public static readonly OptionalParameter<IObserver<IFailedRuntime>> OnFailedRuntime = new OptionalParameter<IObserver<IFailedRuntime>>();
+
+        public static readonly OptionalParameter<IObserver<IWakeError>> OnWakeError = new OptionalParameter<IObserver<IWakeError>>();
+
+        public static ConfigurationModule ConfigurationModule => new ClientConfiguration()
+                    .BindNamedParameter(GenericType<ClientParameters.RunningJobHandler>.Class, OnRunningJob)
+                    .BindNamedParameter(GenericType<ClientParameters.SubmittedJobHandler>.Class, OnSubmittedJob)
+                    .BindNamedParameter(GenericType<ClientParameters.CompletedJobHandler>.Class, OnCompletedJob)
+                    .BindNamedParameter(GenericType<ClientParameters.FailedJobHandler>.Class, OnFailedJob)
+                    .BindNamedParameter(GenericType<ClientParameters.JobMessageHandler>.Class, OnJobMessage)
+                    .BindNamedParameter(GenericType<ClientParameters.FailedRuntimeHandler>.Class, OnFailedRuntime)
+                    .BindNamedParameter(GenericType<ClientParameters.WakeErrorHandler>.Class, OnWakeError)
+                    .Build();
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/ClientParameters.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/ClientParameters.cs
new file mode 100644
index 0000000..edd9f8f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/ClientParameters.cs
@@ -0,0 +1,65 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client.Default;
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Tang.Annotations;
+using System;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config
+{
+    /// <summary>
+    /// Parameters associated with the client.
+    /// </summary>
+    public sealed class ClientParameters
+    {
+        [NamedParameter("completed job handler", defaultClass: typeof(DefaultCompletedJobHandler))]
+        public sealed class CompletedJobHandler : Name<IObserver<ICompletedJob>>
+        {
+        }
+
+        [NamedParameter("failed job handler", defaultClass: typeof(DefaultFailedJobHandler))]
+        public sealed class FailedJobHandler : Name<IObserver<IFailedJob>>
+        {
+        }
+
+        [NamedParameter("failed runtime handler", defaultClass: typeof(DefaultFailedRuntimeHandler))]
+        public sealed class FailedRuntimeHandler : Name<IObserver<IFailedRuntime>>
+        {
+        }
+
+        [NamedParameter("job message handler", defaultClass: typeof(DefaultJobMessageHandler))]
+        public sealed class JobMessageHandler : Name<IObserver<IJobMessage>>
+        {
+        }
+
+        [NamedParameter("running job handler", defaultClass: typeof(DefaultRunningJobHandler))]
+        public sealed class RunningJobHandler : Name<IObserver<IRunningJob>>
+        {
+        }
+
+        [NamedParameter("submitted job handler", defaultClass: typeof(DefaultSubmittedJobHandler))]
+        public sealed class SubmittedJobHandler : Name<IObserver<ISubmittedJob>>
+        {
+        }
+
+        [NamedParameter("wake error handler", defaultClass: typeof(DefaultWakeErrorHandler))]
+        public sealed class WakeErrorHandler : Name<IObserver<IWakeError>>
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverApplicationConfiguration.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverApplicationConfiguration.cs
new file mode 100644
index 0000000..68255a0
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverApplicationConfiguration.cs
@@ -0,0 +1,219 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Driver;
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities.Attributes;
+using Org.Apache.REEF.Wake.Time;
+using System;
+using System.Diagnostics;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config
+{
+    /// <summary>
+    /// Fill this out to configure a Driver.
+    /// </summary>
+    [ClientSide]
+    public sealed class DriverApplicationConfiguration : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        /// The event handler called after the Driver started.
+        /// </summary>
+        public static readonly RequiredImpl<IObserver<IDriverStarted>> OnDriverStarted =
+            new RequiredImpl<IObserver<IDriverStarted>>();
+
+        /// <summary>
+        /// The event handler called when the Driver has been stopped.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IDriverStopped>> OnDriverStopped =
+            new OptionalImpl<IObserver<IDriverStopped>>();
+
+        /// <summary>
+        /// The event handler invoked when driver restarts
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IDriverRestarted>> OnDriverRestarted =
+            new OptionalImpl<IObserver<IDriverRestarted>>();
+
+        /// <summary>
+        /// Event handler for allocated evaluators. Defaults to returning the evaluator if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IAllocatedEvaluator>> OnEvaluatorAllocated =
+            new OptionalImpl<IObserver<IAllocatedEvaluator>>();
+
+        /// <summary>
+        /// Event handler for completed evaluators. Defaults to logging if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<ICompletedEvaluator>> OnEvaluatorCompleted =
+            new OptionalImpl<IObserver<ICompletedEvaluator>>();
+
+        /// <summary>
+        /// Event handler for failed evaluators. Defaults to job failure if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IFailedEvaluator>> OnEvaluatorFailed =
+            new OptionalImpl<IObserver<IFailedEvaluator>>();
+
+        /// <summary>
+        /// Event handler for task messages. Defaults to logging if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<ITaskMessage>> OnTaskMessage =
+            new OptionalImpl<IObserver<ITaskMessage>>();
+
+        /// <summary>
+        /// Event handler for completed tasks. Defaults to closing the context the task ran on if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<ICompletedTask>> OnTaskCompleted =
+            new OptionalImpl<IObserver<ICompletedTask>>();
+
+        /// <summary>
+        /// Event handler for failed tasks. Defaults to job failure if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IFailedTask>> OnTaskFailed =
+            new OptionalImpl<IObserver<IFailedTask>>();
+
+        ///// <summary>
+        ///// Event handler for running tasks. Defaults to logging if not bound.
+        ///// </summary>
+        public static readonly OptionalImpl<IObserver<IRunningTask>> OnTaskRunning =
+            new OptionalImpl<IObserver<IRunningTask>>();
+
+        ///// <summary>
+        ///// Event handler for running task received during driver restart. Defaults to logging if not bound.
+        ///// </summary>
+        public static readonly OptionalImpl<IObserver<IRunningTask>> OnDriverRestartTaskRunning =
+            new OptionalImpl<IObserver<IRunningTask>>();
+
+        /// <summary>
+        /// Event handler for suspended tasks. Defaults to job failure if not bound.
+        /// </summary>
+        /// <remarks>
+        /// Rationale: many jobs don't support task suspension. Hence, this parameter should be optional. The only sane default is
+        /// to crash the job, then.
+        /// </remarks>
+        public static readonly OptionalImpl<IObserver<ISuspendedTask>> OnTaskSuspended =
+            new OptionalImpl<IObserver<ISuspendedTask>>();
+
+        /// <summary>
+        /// Event handler for active context. Defaults to closing the context if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IActiveContext>> OnContextActive =
+            new OptionalImpl<IObserver<IActiveContext>>();
+
+        /// <summary>
+        /// Event handler for active context received during driver restart. Defaults to closing the context if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IActiveContext>> OnDriverRestartContextActive =
+            new OptionalImpl<IObserver<IActiveContext>>();
+
+        /// <summary>
+        /// Event handler for closed context. Defaults to logging if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IClosedContext>> OnContextClosed =
+            new OptionalImpl<IObserver<IClosedContext>>();
+
+        /// <summary>
+        /// Event handler for closed context. Defaults to job failure if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IFailedContext>> OnContextFailed =
+            new OptionalImpl<IObserver<IFailedContext>>();
+
+        /// <summary>
+        /// Event handler for context messages. Defaults to logging if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IContextMessage>> OnContextMessage =
+            new OptionalImpl<IObserver<IContextMessage>>();
+
+        /// <summary>
+        /// Event handler for driver restart completed. Defaults to logging if not bound.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IDriverRestartCompleted>> OnDriverRestartCompleted =
+            new OptionalImpl<IObserver<IDriverRestartCompleted>>();
+
+        ///// <summary>
+        ///// Event handler for driver restart failed evaluator event received during driver restart. Defaults to job failure if not bound.
+        ///// </summary>
+        public static readonly OptionalImpl<IObserver<IFailedEvaluator>> OnDriverRestartEvaluatorFailed =
+            new OptionalImpl<IObserver<IFailedEvaluator>>();
+
+        /// <summary>
+        /// Event handler for client close.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<byte[]>> OnClientClose = new OptionalImpl<IObserver<byte[]>>();
+
+        /// <summary>
+        /// Event handler for client close with message.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<byte[]>> OnClientCloseWithMessage = new OptionalImpl<IObserver<byte[]>>();
+
+        /// <summary>
+        /// Event handler for client message.
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<byte[]>> OnClientMessage = new OptionalImpl<IObserver<byte[]>>();
+
+        /// <summary>
+        /// The trace level of the TraceListener
+        /// </summary>
+        public static readonly OptionalParameter<string> CustomTraceLevel = new OptionalParameter<string>();
+
+        /// <summary>
+        /// Additional set of trace listeners provided by client
+        /// </summary>
+        public static readonly OptionalParameter<TraceListener> CustomTraceListeners =
+            new OptionalParameter<TraceListener>();
+
+        public static ConfigurationModule ConfigurationModule => new DriverApplicationConfiguration()
+            .BindImplementation(GenericType<IClock>.Class, GenericType<BridgeClock>.Class)
+            .BindImplementation(GenericType<IEvaluatorRequestor>.Class,
+                GenericType<DriverBridgeEvaluatorRequestor>.Class)
+
+            // Event handlers
+            .BindSetEntry(GenericType<DriverApplicationParameters.DriverStartedHandlers>.Class, OnDriverStarted)
+            .BindSetEntry(GenericType<DriverApplicationParameters.DriverStopHandlers>.Class, OnDriverStopped)
+            .BindSetEntry(GenericType<DriverApplicationParameters.AllocatedEvaluatorHandlers>.Class, OnEvaluatorAllocated)
+            .BindSetEntry(GenericType<DriverApplicationParameters.ActiveContextHandlers>.Class, OnContextActive)
+            .BindSetEntry(GenericType<DriverApplicationParameters.TaskMessageHandlers>.Class, OnTaskMessage)
+            .BindSetEntry(GenericType<DriverApplicationParameters.FailedTaskHandlers>.Class, OnTaskFailed)
+            .BindSetEntry(GenericType<DriverApplicationParameters.RunningTaskHandlers>.Class, OnTaskRunning)
+            .BindSetEntry(GenericType<DriverApplicationParameters.SuspendedTaskHandlers>.Class, OnTaskSuspended)
+            .BindSetEntry(GenericType<DriverApplicationParameters.FailedEvaluatorHandlers>.Class, OnEvaluatorFailed)
+            .BindSetEntry(GenericType<DriverApplicationParameters.CompletedEvaluatorHandlers>.Class, OnEvaluatorCompleted)
+            .BindSetEntry(GenericType<DriverApplicationParameters.CompletedTaskHandlers>.Class, OnTaskCompleted)
+            .BindSetEntry(GenericType<DriverApplicationParameters.ClosedContextHandlers>.Class, OnContextClosed)
+            .BindSetEntry(GenericType<DriverApplicationParameters.FailedContextHandlers>.Class, OnContextFailed)
+            .BindSetEntry(GenericType<DriverApplicationParameters.ContextMessageHandlers>.Class, OnContextMessage)
+            .BindSetEntry(GenericType<DriverApplicationParameters.DriverRestartCompletedHandlers>.Class,
+                OnDriverRestartCompleted)
+            .BindSetEntry(GenericType<DriverApplicationParameters.DriverRestartedHandlers>.Class, OnDriverRestarted)
+            .BindSetEntry(GenericType<DriverApplicationParameters.DriverRestartActiveContextHandlers>.Class,
+                OnDriverRestartContextActive)
+            .BindSetEntry(GenericType<DriverApplicationParameters.DriverRestartRunningTaskHandlers>.Class,
+                OnDriverRestartTaskRunning)
+            .BindSetEntry(GenericType<DriverApplicationParameters.DriverRestartFailedEvaluatorHandlers>.Class,
+                OnDriverRestartEvaluatorFailed)
+            .BindSetEntry(GenericType<DriverApplicationParameters.ClientCloseHandlers>.Class, OnClientClose)
+            .BindSetEntry(GenericType<DriverApplicationParameters.ClientCloseWithMessageHandlers>.Class, OnClientCloseWithMessage)
+            .BindSetEntry(GenericType<DriverApplicationParameters.ClientMessageHandlers>.Class, OnClientMessage)
+            .BindSetEntry(GenericType<DriverApplicationParameters.TraceListeners>.Class, CustomTraceListeners)
+            .BindNamedParameter(GenericType<DriverApplicationParameters.TraceLevel>.Class, CustomTraceLevel)
+            .Build();
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverApplicationParameters.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverApplicationParameters.cs
new file mode 100644
index 0000000..df72f50
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverApplicationParameters.cs
@@ -0,0 +1,164 @@
+// 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.
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Bridge;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Defaults;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Annotations;
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config
+{
+    public sealed class DriverApplicationParameters
+    {
+        [NamedParameter(documentation: "The start point for application logic. Event fired after the Driver is done initializing.")]
+        public sealed class DriverStartedHandlers : Name<ISet<IObserver<IDriverStarted>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when driver stops.")]
+        public sealed class DriverStopHandlers : Name<ISet<IObserver<IDriverStopped>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when an evaluator completes.", defaultClasses: new[] { typeof(DefaultEvaluatorCompletionHandler) })]
+        public class CompletedEvaluatorHandlers : Name<ISet<IObserver<ICompletedEvaluator>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when an allocated evaluator is given to the client.",
+            defaultClasses: new[] { typeof(DefaultEvaluatorAllocationHandler) })]
+        public sealed class AllocatedEvaluatorHandlers : Name<ISet<IObserver<IAllocatedEvaluator>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when an exception occurs on a running evaluator.", defaultClasses: new[] { typeof(DefaultEvaluatorFailureHandler) })]
+        public class FailedEvaluatorHandlers : Name<ISet<IObserver<IFailedEvaluator>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for IActiveContext.", defaultClasses: new[] { typeof(DefaultContextActiveHandler) })]
+        public sealed class ActiveContextHandlers : Name<ISet<IObserver<IActiveContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for ClosedContext.", defaultClasses: new[] { typeof(DefaultContextClosureHandler) })]
+        public sealed class ClosedContextHandlers : Name<ISet<IObserver<IClosedContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for ContextMessage.", defaultClasses: new[] { typeof(DefaultContextMessageHandler) })]
+        public sealed class ContextMessageHandlers : Name<ISet<IObserver<IContextMessage>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for FailedContext.", defaultClasses: new[] { typeof(DefaultContextFailureHandler) })]
+        public sealed class FailedContextHandlers : Name<ISet<IObserver<IFailedContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Running task handler.", defaultClasses: new[] { typeof(DefaultTaskRunningHandler) })]
+        public sealed class RunningTaskHandlers : Name<ISet<IObserver<IRunningTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Completed task handler.", defaultClasses: new[] { typeof(DefaultTaskCompletionHandler) })]
+        public sealed class CompletedTaskHandlers : Name<ISet<IObserver<ICompletedTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Task exception handler.", defaultClasses: new[] { typeof(DefaultTaskFailureHandler) })]
+        public sealed class FailedTaskHandlers : Name<ISet<IObserver<IFailedTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Suspended task handler.", defaultClasses: new[] { typeof(DefaultTaskSuspensionHandler) })]
+        public sealed class SuspendedTaskHandlers : Name<ISet<IObserver<ISuspendedTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Task message handler.", defaultClasses: new[] { typeof(DefaultTaskMessageHandler) })]
+        public class TaskMessageHandlers : Name<ISet<IObserver<ITaskMessage>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when driver is restarted, after CLR bridge is set up.")]
+        public sealed class DriverRestartedHandlers : Name<ISet<IObserver<IDriverRestarted>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when an evaluator has failed in the Driver Restart process.",
+            defaultClasses: new[] { typeof(DefaultEvaluatorFailureHandler) })]
+        public sealed class DriverRestartFailedEvaluatorHandlers : Name<ISet<IObserver<IFailedEvaluator>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for IActiveContext received during driver restart.",
+            defaultClasses: new[] { typeof(DefaultDriverRestartContextActiveHandler) })]
+        public sealed class DriverRestartActiveContextHandlers : Name<ISet<IObserver<IActiveContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when driver restart is completed.",
+            defaultClasses: new[] { typeof(DefaultDriverRestartCompletedHandler) })]
+        public sealed class DriverRestartCompletedHandlers : Name<ISet<IObserver<IDriverRestartCompleted>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Running task during driver restart handler.",
+            defaultClasses: new[] { typeof(DefaultDriverRestartTaskRunningHandler) })]
+        public sealed class DriverRestartRunningTaskHandlers : Name<ISet<IObserver<IRunningTask>>>
+        {
+        }
+
+        [NamedParameter("Client message handlers", defaultClasses: new[] { typeof(DefaultClientMessageHandler) })]
+        public sealed class ClientMessageHandlers : Name<ISet<IObserver<byte[]>>>
+        {
+        }
+
+        [NamedParameter("Client close with message event handlers", defaultClasses: new[] { typeof(DefaultClientCloseWithMessageHandler) })]
+        public sealed class ClientCloseWithMessageHandlers : Name<ISet<IObserver<byte[]>>>
+        {
+        }
+
+        [NamedParameter("Client close event handlers", defaultClasses: new[] { typeof(DefaultClientCloseHandler) })]
+        public sealed class ClientCloseHandlers : Name<ISet<IObserver<byte[]>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Http Event Handlers.", defaultClasses: new[] { typeof(DefaultHttpHandler) })]
+        public sealed class HttpEventHandlers : Name<ISet<IHttpHandler>>
+        {
+        }
+
+        [NamedParameter("Custom Trace Level", "DriverTraceLevel", defaultValue: "Info")]
+        public class TraceLevel : Name<string>
+        {
+        }
+
+        [NamedParameter("Additional trace listeners supplied by client", "DriverTraceListeners", null,
+            defaultClasses: new[] { typeof(DefaultCustomTraceListener) })]
+        public sealed class TraceListeners : Name<ISet<TraceListener>>
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverRuntimeConfiguration.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverRuntimeConfiguration.cs
new file mode 100644
index 0000000..b3428f8
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverRuntimeConfiguration.cs
@@ -0,0 +1,148 @@
+// 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.
+
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+using System;
+using System.IO;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config
+{
+    /// <summary>
+    /// Configuration Module for the driver runtime, which corresponds to the
+    /// Java driver.
+    /// </summary>
+    public sealed class DriverRuntimeConfiguration : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        /// The Job identifier.
+        /// </summary>
+        public static readonly RequiredParameter<string> JobId = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Job submission directory where related job files are staged for submission.
+        /// </summary>
+        public static readonly OptionalParameter<string> JobSubmissionDirectory = new OptionalParameter<string>();
+
+        /// <summary>
+        /// How many CPU cores should be dedicated to the driver.
+        /// </summary>
+        public static readonly OptionalParameter<int> DriverCpuCores = new OptionalParameter<int>();
+
+        /// <summary>
+        /// How much memory in MegaBytes should be dedicated to the driver.
+        /// </summary>
+        public static readonly OptionalParameter<int> DriverMemory = new OptionalParameter<int>();
+
+        /// <summary>
+        /// Assembiles that should be present on the driver and all evaluators.
+        /// </summary>
+        public static readonly OptionalParameter<string> GlobalAssemblies = new OptionalParameter<string>();
+
+        /// <summary>
+        /// Assembiles that should be present only on the driver.
+        /// </summary>
+        public static readonly OptionalParameter<string> LocalAssemblies = new OptionalParameter<string>();
+
+        /// <summary>
+        /// Files that should be present on the driver and all evaluators.
+        /// </summary>
+        public static readonly OptionalParameter<string> GlobalFiles = new OptionalParameter<string>();
+
+        /// <summary>
+        /// Files that should be present only on the driver.
+        /// </summary>
+        public static readonly OptionalParameter<string> LocalFiles = new OptionalParameter<string>();
+
+        /// <summary>
+        /// Indicates whether support for driver restart should be enabled.
+        /// </summary>
+        public static readonly OptionalParameter<bool> EnableDriverRestart = new OptionalParameter<bool>();
+
+        /// <summary>
+        /// How long should we wait for evaluators to checkin with the restarted driver before
+        /// considering them dead.
+        /// </summary>
+        public static readonly OptionalParameter<int> RestartEvaluatorRecoverySeconds = new OptionalParameter<int>();
+
+        /// <summary>
+        /// The start TCP port to use for driver evaluator communication.
+        /// </summary>
+        public static readonly OptionalParameter<int> TcpPortRangeBegin = new OptionalParameter<int>();
+
+        /// <summary>
+        /// The port range for driver evaluator communication.
+        /// </summary>
+        public static readonly OptionalParameter<int> TcpPortRangeCount = new OptionalParameter<int>();
+
+        /// <summary>
+        /// How many ports should we try before giving up.
+        /// </summary>
+        public static readonly OptionalParameter<int> TcpPortRangeTryCount = new OptionalParameter<int>();
+
+        /// <summary>
+        /// The intended operating system on which the driver and evaluators will run.
+        /// </summary>
+        public static readonly OptionalImpl<OsType> OsType = new OptionalImpl<OsType>();
+
+        public static ConfigurationModule ConfigurationModule => new DriverRuntimeConfiguration()
+            .BindImplementation(GenericType<OsType>.Class, OsType)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.JobId>.Class, JobId)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.JobSubmissionDirectory>.Class, JobSubmissionDirectory)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.DriverCpuCores>.Class, DriverCpuCores)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.DriverMemory>.Class, DriverMemory)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.EnableDriverRestart>.Class, EnableDriverRestart)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.RestartEvaluatorRecoverySeconds>.Class, RestartEvaluatorRecoverySeconds)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.TcpPortRangeBegin>.Class, TcpPortRangeBegin)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.TcpPortRangeCount>.Class, TcpPortRangeCount)
+            .BindNamedParameter(GenericType<DriverRuntimeParameters.TcpPortRangeTryCount>.Class, TcpPortRangeTryCount)
+            .BindSetEntry(GenericType<DriverRuntimeParameters.GlobalAssemblies>.Class, GlobalAssemblies)
+            .BindSetEntry(GenericType<DriverRuntimeParameters.LocalAssemblies>.Class, LocalAssemblies)
+            .BindSetEntry(GenericType<DriverRuntimeParameters.GlobalFiles>.Class, GlobalFiles)
+            .BindSetEntry(GenericType<DriverRuntimeParameters.LocalFiles>.Class, LocalFiles)
+            .Build();
+
+        /// <summary>
+        /// Add the assembly needed for the given Type to the driver.
+        /// </summary>
+        /// <param name="type"></param>
+        /// <returns></returns>
+        public static ConfigurationModule AddLocalAssemblyForType(ConfigurationModule conf, Type type)
+        {
+            return conf.Set(LocalAssemblies, GetAssemblyPathForType(type));
+        }
+
+        /// <summary>
+        /// Add the assembly needed for the given Type to all containers.
+        /// </summary>
+        /// <param name="type"></param>
+        /// <returns></returns>
+        public static ConfigurationModule AddGlobalAssemblyForType(ConfigurationModule conf, Type type)
+        {
+            return conf.Set(GlobalAssemblies, GetAssemblyPathForType(type));
+        }
+
+        /// <summary>
+        /// Finds the path to the assembly the given Type was loaded from.
+        /// </summary>
+        private static string GetAssemblyPathForType(Type type)
+        {
+            var path = Uri.UnescapeDataString(new UriBuilder(type.Assembly.CodeBase).Path);
+            return Path.GetFullPath(path);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverRuntimeParameters.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverRuntimeParameters.cs
new file mode 100644
index 0000000..cb295ce
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/DriverRuntimeParameters.cs
@@ -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.
+using Org.Apache.REEF.Tang.Annotations;
+using System.Collections.Generic;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config
+{
+    public sealed class DriverRuntimeParameters
+    {
+        [NamedParameter("the job identifier")]
+        public sealed class JobId : Name<string>
+        {
+        }
+
+        [NamedParameter("The job submission directory", defaultValue: "")]
+        public sealed class JobSubmissionDirectory : Name<string>
+        {
+        }
+
+        [NamedParameter("Number of cpu cores", defaultValue: "1")]
+        public sealed class DriverCpuCores : Name<int>
+        {
+        }
+
+        [NamedParameter("The amount of driver memory in MB", defaultValue: "512")]
+        public sealed class DriverMemory : Name<int>
+        {
+        }
+
+        [NamedParameter("enable driver restart", defaultValue: "true")]
+        public sealed class EnableDriverRestart : Name<bool>
+        {
+        }
+
+        [NamedParameter("the amount of time to wait for evaluators to recover after failed driver", defaultValue: "0")]
+        public sealed class RestartEvaluatorRecoverySeconds : Name<int>
+        {
+        }
+
+        [NamedParameter("Driver and Evaluator assemblies.")]
+        public sealed class GlobalAssemblies : Name<ISet<string>>
+        {
+        }
+
+        [NamedParameter("Driver only related assemblies.")]
+        public sealed class LocalAssemblies : Name<ISet<string>>
+        {
+        }
+
+        [NamedParameter("Driver and Evaluator file resources.")]
+        public sealed class GlobalFiles : Name<ISet<string>>
+        {
+        }
+
+        [NamedParameter("Driver only related file resources.")]
+        public sealed class LocalFiles : Name<ISet<string>>
+        {
+        }
+
+        [NamedParameter("tcp port range begin", defaultValue: "0")]
+        public sealed class TcpPortRangeBegin : Name<int>
+        {
+        }
+
+        [NamedParameter("tcp port range count", defaultValue: "0")]
+        public sealed class TcpPortRangeCount : Name<int>
+        {
+        }
+
+        [NamedParameter("tcp port range try count", defaultValue: "0")]
+        public sealed class TcpPortRangeTryCount : Name<int>
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/OsType.cs
similarity index 59%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/OsType.cs
index 2f3352a..3761da5 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/OsType.cs
@@ -17,10 +17,36 @@
 
 using Org.Apache.REEF.Tang.Annotations;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    [DefaultImplementation(typeof(OsWindows))]
+    public abstract class OsType
     {
+        public enum Os
+        {
+            Windows, Linux
+        }
+
+        public abstract Os Type { get; }
+    }
+
+    public sealed class OsLinux : OsType
+    {
+        [Inject]
+        private OsLinux()
+        {
+        }
+
+        public override Os Type => Os.Linux;
+    }
+
+    public sealed class OsWindows : OsType
+    {
+        [Inject]
+        private OsWindows()
+        {
+        }
+
+        public override Os Type => Os.Windows;
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/AzureBatchRuntimeConfiguration.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/AzureBatchRuntimeConfiguration.cs
new file mode 100644
index 0000000..cab65fb
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/AzureBatchRuntimeConfiguration.cs
@@ -0,0 +1,85 @@
+// 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.
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    public sealed class AzureBatchRuntimeConfiguration : ConfigurationModuleBuilder
+    {
+        private static readonly string AzBatchConfigurationFileEnvironmentVariable = "REEF_AZBATCH_CONF";
+
+        /// <summary>
+        /// The Azure Batch account URI.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureBatchAccountUri = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Batch account name.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureBatchAccountName = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Batch account key.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureBatchAccountKey = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Batch pool ID.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureBatchPoolId = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Storage account name.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureStorageAccountName = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Storage account key.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureStorageAccountKey = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Storage container name.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureStorageContainerName = new RequiredParameter<string>();
+
+        public static ConfigurationModule ConfigurationModule = new AzureBatchRuntimeConfiguration()
+            .BindImplementation(GenericType<IRuntimeProtoProvider>.Class, GenericType<AzureBatchRuntimeProtoProvider>.Class)
+            .BindNamedParameter(GenericType<AzureBatchRuntimeParameters.AzureBatchAccountUri>.Class,
+                AzureBatchAccountUri)
+            .BindNamedParameter(GenericType<AzureBatchRuntimeParameters.AzureBatchAccountName>.Class,
+                AzureBatchAccountName)
+            .BindNamedParameter(GenericType<AzureBatchRuntimeParameters.AzureBatchAccountKey>.Class,
+                AzureBatchAccountKey)
+            .BindNamedParameter(GenericType<AzureBatchRuntimeParameters.AzureBatchPoolId>.Class, AzureBatchPoolId)
+            .BindNamedParameter(GenericType<AzureBatchRuntimeParameters.AzureStorageAccountName>.Class,
+                AzureStorageAccountName)
+            .BindNamedParameter(GenericType<AzureBatchRuntimeParameters.AzureStorageAccountKey>.Class,
+                AzureStorageAccountKey)
+            .BindNamedParameter(GenericType<AzureBatchRuntimeParameters.AzureStorageContainerName>.Class,
+                AzureStorageContainerName)
+            .Build();
+
+        public static IConfiguration FromEnvironment()
+        {
+            return Utils.FromEnvironment(AzBatchConfigurationFileEnvironmentVariable);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/AzureBatchRuntimeParameters.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/AzureBatchRuntimeParameters.cs
new file mode 100644
index 0000000..2dec1d4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/AzureBatchRuntimeParameters.cs
@@ -0,0 +1,58 @@
+// 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.
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    internal sealed class AzureBatchRuntimeParameters
+    {
+        [NamedParameter(Documentation = "The Azure Batch Account Key")]
+        public sealed class AzureBatchAccountKey : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Batch Account Name")]
+        public sealed class AzureBatchAccountName : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Batch account URI")]
+        public sealed class AzureBatchAccountUri : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Batch Pool Id")]
+        public sealed class AzureBatchPoolId : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Storage Account Key")]
+        public sealed class AzureStorageAccountKey : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Storage Account Name")]
+        public sealed class AzureStorageAccountName : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Storage Container Name")]
+        public sealed class AzureStorageContainerName : Name<string>
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/HdInsightRuntimeConfiguration.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/HdInsightRuntimeConfiguration.cs
new file mode 100644
index 0000000..ce80559
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/HdInsightRuntimeConfiguration.cs
@@ -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.
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    public sealed class HdInsightRuntimeConfiguration : ConfigurationModuleBuilder
+    {
+        private static readonly string HdiConfigurationFileEnvironmentVariable = "REEF_HDI_CONF";
+
+        /// <summary>
+        /// HDInsight URL.
+        /// </summary>
+        public static readonly RequiredParameter<string> HdInsightUrl = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The user name.
+        /// </summary>
+        public static readonly RequiredParameter<string> HdInsightUserName = new RequiredParameter<string>();
+
+        /// <summary>
+        /// Password.
+        /// </summary>
+        public static readonly RequiredParameter<string> HdInsightPassword = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Storage account name.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureStorageAccountName = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Storage account key.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureStorageAccountKey = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Azure Storage container name.
+        /// </summary>
+        public static readonly RequiredParameter<string> AzureStorageContainerName = new RequiredParameter<string>();
+
+        /// <summary>
+        /// Indicates whether to target an unsafe deployment.
+        /// </summary>
+        public static readonly OptionalParameter<bool> HdInsightUnsafeDeployment = new OptionalParameter<bool>();
+
+        public static ConfigurationModule ConfigurationModule = new HdInsightRuntimeConfiguration()
+            .BindImplementation(GenericType<IRuntimeProtoProvider>.Class, GenericType<HdInsightRuntimeProtoProvider>.Class)
+            .BindNamedParameter(GenericType<HdInsightRuntimeParameters.HdInsightUrl>.Class, HdInsightUrl)
+            .BindNamedParameter(GenericType<HdInsightRuntimeParameters.HdInsightUserName>.Class, HdInsightUserName)
+            .BindNamedParameter(GenericType<HdInsightRuntimeParameters.HdInsightPassword>.Class, HdInsightPassword)
+            .BindNamedParameter(GenericType<HdInsightRuntimeParameters.AzureStorageAccountName>.Class,
+                AzureStorageAccountName)
+            .BindNamedParameter(GenericType<HdInsightRuntimeParameters.AzureStorageAccountKey>.Class,
+                AzureStorageAccountKey)
+            .BindNamedParameter(GenericType<HdInsightRuntimeParameters.AzureStorageContainerName>.Class,
+                AzureStorageContainerName)
+            .BindNamedParameter(GenericType<HdInsightRuntimeParameters.HdInsightUnsafeDeployment>.Class, HdInsightUnsafeDeployment)
+            .Build();
+
+        public static IConfiguration FromEnvironment()
+        {
+            return Utils.FromEnvironment(HdiConfigurationFileEnvironmentVariable);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/HdInsightRuntimeParameters.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/HdInsightRuntimeParameters.cs
new file mode 100644
index 0000000..190c96a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/HdInsightRuntimeParameters.cs
@@ -0,0 +1,58 @@
+// 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.
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    internal sealed class HdInsightRuntimeParameters
+    {
+        [NamedParameter(Documentation = "Unsafe deployment of HDInsight", DefaultValue = "false")]
+        public sealed class HdInsightUnsafeDeployment : Name<bool>
+        {
+        }
+
+        [NamedParameter(Documentation = "HDInsight Password")]
+        public sealed class HdInsightPassword : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "HDInsight User Name")]
+        public sealed class HdInsightUserName : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "HDInsight URL")]
+        public sealed class HdInsightUrl : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Storage Account Key")]
+        public sealed class AzureStorageAccountKey : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Storage Account Name")]
+        public sealed class AzureStorageAccountName : Name<string>
+        {
+        }
+
+        [NamedParameter(Documentation = "The Azure Storage Container Name")]
+        public sealed class AzureStorageContainerName : Name<string>
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/LocalRuntimeConfiguration.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/LocalRuntimeConfiguration.cs
new file mode 100644
index 0000000..f0b086e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/LocalRuntimeConfiguration.cs
@@ -0,0 +1,51 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    public sealed class LocalRuntimeConfiguration : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        /// The number of threads or evaluators available to the resourcemanager.
+        /// </summary>
+        /// <remarks>
+        /// This is the upper limit on the number of
+        /// Evaluators that the local resourcemanager will hand out concurrently. This simulates the size of a physical cluster
+        /// in terms of the number of slots available on it with one important caveat: The Driver is not counted against this
+        /// number.
+        /// </remarks>
+        public static readonly OptionalParameter<int> NumberOfEvaluators = new OptionalParameter<int>();
+
+        /// <summary>
+        /// The folder in which the sub-folders, one per job, will be created.
+        /// </summary>
+        /// <remarks>
+        /// If none is given, the temp directory is used.
+        /// </remarks>
+        public static readonly OptionalParameter<string> RuntimeFolder = new OptionalParameter<string>();
+
+        public static ConfigurationModule ConfigurationModule = new LocalRuntimeConfiguration()
+            .BindImplementation(GenericType<IRuntimeProtoProvider>.Class, GenericType<LocalRuntimeProtoProvider>.Class)
+            .BindNamedParameter(GenericType<LocalRuntimeParameters.LocalRuntimeDirectory>.Class, RuntimeFolder)
+            .BindNamedParameter(GenericType<LocalRuntimeParameters.NumberOfEvaluators>.Class, NumberOfEvaluators)
+            .Build();
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/LocalRuntimeParameters.cs
similarity index 66%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/LocalRuntimeParameters.cs
index 2f3352a..04af5fe 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/LocalRuntimeParameters.cs
@@ -14,13 +14,20 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
 using Org.Apache.REEF.Tang.Annotations;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class LocalRuntimeParameters
     {
+        [NamedParameter("The directory in which the local runtime will store its execution.", defaultValue: "")]
+        public class LocalRuntimeDirectory : Name<string>
+        {
+        }
+
+        [NamedParameter(defaultValue: "2")]
+        public class NumberOfEvaluators : Name<int>
+        {
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/AzureBatchRuntimeProtoProvider.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/AzureBatchRuntimeProtoProvider.cs
new file mode 100644
index 0000000..73b6d4f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/AzureBatchRuntimeProtoProvider.cs
@@ -0,0 +1,53 @@
+// 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.
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto
+{
+    internal sealed class AzureBatchRuntimeProtoProvider : IRuntimeProtoProvider
+    {
+        private readonly Core.Proto.AzureBatchRuntimeParameters _azureBatchRuntimeParameters;
+
+        [Inject]
+        private AzureBatchRuntimeProtoProvider(
+            [Parameter(Value = typeof(AzureBatchRuntimeParameters.AzureBatchAccountKey))] string accountKey,
+            [Parameter(Value = typeof(AzureBatchRuntimeParameters.AzureBatchAccountName))] string accountName,
+            [Parameter(Value = typeof(AzureBatchRuntimeParameters.AzureBatchAccountUri))] string accountUri,
+            [Parameter(Value = typeof(AzureBatchRuntimeParameters.AzureBatchPoolId))] string poolId,
+            [Parameter(Value = typeof(AzureBatchRuntimeParameters.AzureStorageAccountKey))] string storageAccountKey,
+            [Parameter(Value = typeof(AzureBatchRuntimeParameters.AzureStorageAccountName))] string storageAccountName,
+            [Parameter(Value = typeof(AzureBatchRuntimeParameters.AzureStorageContainerName))] string storageContainerName)
+        {
+            _azureBatchRuntimeParameters = new Core.Proto.AzureBatchRuntimeParameters()
+            {
+                AzureBatchAccountKey = accountKey,
+                AzureBatchAccountName = accountName,
+                AzureBatchAccountUri = accountUri,
+                AzureBatchPoolId = poolId,
+                AzureStorageAccountKey = storageAccountKey,
+                AzureStorageAccountName = storageAccountName,
+                AzureStorageContainerName = storageContainerName
+            };
+        }
+
+        public void SetParameters(DriverClientConfiguration driverClientConfiguration)
+        {
+            driverClientConfiguration.AzbatchRuntime = _azureBatchRuntimeParameters;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/HdInsightRuntimeProtoProvider.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/HdInsightRuntimeProtoProvider.cs
new file mode 100644
index 0000000..7fb259f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/HdInsightRuntimeProtoProvider.cs
@@ -0,0 +1,53 @@
+// 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.
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto
+{
+    internal sealed class HdInsightRuntimeProtoProvider : IRuntimeProtoProvider
+    {
+        private readonly HDIRuntimeParameters _hdiRuntimeParameters;
+
+        [Inject]
+        private HdInsightRuntimeProtoProvider(
+            [Parameter(Value = typeof(HdInsightRuntimeParameters.HdInsightUnsafeDeployment))] bool unsafeDeplyment,
+            [Parameter(Value = typeof(HdInsightRuntimeParameters.HdInsightUserName))] string userName,
+            [Parameter(Value = typeof(HdInsightRuntimeParameters.HdInsightPassword))] string password,
+            [Parameter(Value = typeof(HdInsightRuntimeParameters.HdInsightUrl))] string url,
+            [Parameter(Value = typeof(HdInsightRuntimeParameters.AzureStorageAccountKey))] string storageAccountKey,
+            [Parameter(Value = typeof(HdInsightRuntimeParameters.AzureStorageAccountName))] string storageAccountName,
+            [Parameter(Value = typeof(HdInsightRuntimeParameters.AzureStorageContainerName))] string storageContainerName)
+        {
+            _hdiRuntimeParameters = new HDIRuntimeParameters()
+            {
+                Unsafe = unsafeDeplyment,
+                HdiUserName = userName,
+                HdiPassword = password,
+                HdiUrl = url,
+                AzureStorageAccountKey = storageAccountKey,
+                AzureStorageAccountName = storageAccountName,
+                AzureStorageContainerName = storageContainerName
+            };
+        }
+
+        public void SetParameters(DriverClientConfiguration driverClientConfiguration)
+        {
+            driverClientConfiguration.HdiRuntime = _hdiRuntimeParameters;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/IRuntimeProtoProvider.cs
similarity index 75%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/IRuntimeProtoProvider.cs
index 2f3352a..334f3ea 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/IRuntimeProtoProvider.cs
@@ -15,12 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Bridge.Core.Proto;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal interface IRuntimeProtoProvider
     {
+        void SetParameters(DriverClientConfiguration driverClientConfiguration);
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/LocalRuntimeProtoProvider.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/LocalRuntimeProtoProvider.cs
new file mode 100644
index 0000000..5dc212f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/LocalRuntimeProtoProvider.cs
@@ -0,0 +1,49 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto
+{
+    internal sealed class LocalRuntimeProtoProvider : IRuntimeProtoProvider
+    {
+        private readonly Core.Proto.LocalRuntimeParameters _localRuntimeParameters;
+
+        [Inject]
+        private LocalRuntimeProtoProvider(
+            [Parameter(Value = typeof(LocalRuntimeParameters.LocalRuntimeDirectory))] string localRuntimeDirectory,
+            [Parameter(Value = typeof(LocalRuntimeParameters.NumberOfEvaluators))] int numberOfEvaluators)
+        {
+            if (numberOfEvaluators <= 0)
+            {
+                throw new IllegalStateException("Number of evaluators must be greater than zero");
+            }
+            _localRuntimeParameters = new Core.Proto.LocalRuntimeParameters()
+            {
+                MaxNumberOfEvaluators = (uint)numberOfEvaluators,
+                RuntimeRootFolder = localRuntimeDirectory
+            };
+        }
+
+        public void SetParameters(DriverClientConfiguration driverClientConfiguration)
+        {
+            driverClientConfiguration.LocalRuntime = _localRuntimeParameters;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/YarnRuntimeProtoProvider.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/YarnRuntimeProtoProvider.cs
new file mode 100644
index 0000000..0b9adad
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Proto/YarnRuntimeProtoProvider.cs
@@ -0,0 +1,57 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using System.Collections.Generic;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto
+{
+    internal sealed class YarnRuntimeProtoProvider : IRuntimeProtoProvider
+    {
+        private readonly Core.Proto.YarnRuntimeParameters _yarnRuntimeParameters;
+
+        [Inject]
+        private YarnRuntimeProtoProvider(
+            [Parameter(Value = typeof(YarnRuntimeParameters.JobPriority))] int jobPriority,
+            [Parameter(Value = typeof(YarnRuntimeParameters.JobQueue))] string jobQueue,
+            [Parameter(Value = typeof(YarnRuntimeParameters.JobSubmissionDirectoryPrefix))] string jobSubmissionDirectoryPrefix,
+            [Parameter(Value = typeof(YarnRuntimeParameters.SecurityTokenStrings))] ISet<string> securityTokenStrings,
+            [Parameter(Value = typeof(YarnRuntimeParameters.UnmanagedDriver))] bool unmanagedDriver,
+            [Parameter(Value = typeof(YarnRuntimeParameters.FileSystemUrl))] string fileSystemUrl)
+        {
+            if (jobPriority < 0)
+            {
+                throw new IllegalStateException("Job Priority must be greater than or equal to zero");
+            }
+            _yarnRuntimeParameters = new Core.Proto.YarnRuntimeParameters()
+            {
+                Priority = (uint)jobPriority,
+                Queue = jobQueue,
+                JobSubmissionDirectoryPrefix = jobSubmissionDirectoryPrefix,
+                UnmangedDriver = unmanagedDriver,
+                FilesystemUrl = fileSystemUrl
+            };
+        }
+
+        public void SetParameters(DriverClientConfiguration driverClientConfiguration)
+        {
+            driverClientConfiguration.YarnRuntime = _yarnRuntimeParameters;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Utils.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Utils.cs
new file mode 100644
index 0000000..1935f33
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/Utils.cs
@@ -0,0 +1,47 @@
+// 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.
+
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
+using System;
+using System.IO;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    internal static class Utils
+    {
+        public static IConfiguration FromTextFile(string file)
+        {
+            return new AvroConfigurationSerializer().FromFile(file);
+        }
+
+        public static IConfiguration FromEnvironment(string environmentVariable)
+        {
+            var configurationPath = Environment.GetEnvironmentVariable(environmentVariable);
+            if (configurationPath == null)
+            {
+                throw new ArgumentException($"Environment Variable {environmentVariable} not set");
+            }
+
+            if (!File.Exists(configurationPath))
+            {
+                throw new ArgumentException($"File located by Environment Variable {environmentVariable} cannot be read.");
+            }
+            return FromTextFile(configurationPath);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/YarnRuntimeConfiguration.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/YarnRuntimeConfiguration.cs
new file mode 100644
index 0000000..53e5acd
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/YarnRuntimeConfiguration.cs
@@ -0,0 +1,49 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    public sealed class YarnRuntimeConfiguration : ConfigurationModuleBuilder
+    {
+        public static readonly OptionalParameter<int> JobPriority = new OptionalParameter<int>();
+        public static readonly OptionalParameter<string> JobQueue = new OptionalParameter<string>();
+        public static readonly OptionalParameter<string> JobSubmissionDirectoryPrefix = new OptionalParameter<string>();
+        public static readonly OptionalParameter<string> SecurityTokenStrings = new OptionalParameter<string>();
+        public static readonly OptionalParameter<bool> UnmanagedDriver = new OptionalParameter<bool>();
+
+        /// <summary>
+        /// URL for store. For Hadoop file system, it is set in fs.defaultFS as default by YARN environment. Client doesn't need to
+        /// specify it. For Data Lake, Yarn applications are required to set the complete path by themselves
+        /// e.g. adl://reefadl.azuredatalakestore.net
+        /// </summary>
+        public static readonly OptionalParameter<string> FileSystemUrl = new OptionalParameter<string>();
+
+        public static ConfigurationModule ConfigurationModule = new YarnRuntimeConfiguration()
+            .BindImplementation(GenericType<IRuntimeProtoProvider>.Class, GenericType<YarnRuntimeProtoProvider>.Class)
+            .BindNamedParameter(GenericType<YarnRuntimeParameters.JobPriority>.Class, JobPriority)
+            .BindNamedParameter(GenericType<YarnRuntimeParameters.JobSubmissionDirectoryPrefix>.Class, JobSubmissionDirectoryPrefix)
+            .BindNamedParameter(GenericType<YarnRuntimeParameters.JobQueue>.Class, JobQueue)
+            .BindSetEntry(GenericType<YarnRuntimeParameters.SecurityTokenStrings>.Class, SecurityTokenStrings)
+            .BindNamedParameter(GenericType<YarnRuntimeParameters.FileSystemUrl>.Class, FileSystemUrl)
+            .BindNamedParameter(GenericType<YarnRuntimeParameters.UnmanagedDriver>.Class, UnmanagedDriver)
+            .Build();
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/YarnRuntimeParameters.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/YarnRuntimeParameters.cs
new file mode 100644
index 0000000..048c1bb
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Config/Runtime/YarnRuntimeParameters.cs
@@ -0,0 +1,57 @@
+// 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.
+using Org.Apache.REEF.Tang.Annotations;
+using System.Collections.Generic;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime
+{
+    internal sealed class YarnRuntimeParameters
+    {
+        [NamedParameter("FileSystem URL.", defaultValue: "")]
+        public sealed class FileSystemUrl : Name<string>
+        {
+        }
+
+        [NamedParameter("The job priority", defaultValue: "0")]
+        public sealed class JobPriority : Name<int>
+        {
+        }
+
+        [NamedParameter("The job queue.", defaultValue: "")]
+        public sealed class JobQueue : Name<string>
+        {
+        }
+
+        [NamedParameter("Directory used to upload job resources from the client", defaultValue: @"/vol1/tmp")]
+        public class JobSubmissionDirectoryPrefix : Name<string>
+        {
+        }
+
+        /// <summary>
+        /// Named parameter that contains a set of serialized tokens.
+        /// </summary>
+        [NamedParameter("Serialized SurityToken Info", defaultValue: "")]
+        public class SecurityTokenStrings : Name<ISet<string>>
+        {
+        }
+
+        [NamedParameter("Boolean indicating whether we want an unmanged driver.", defaultValue: "false")]
+        public sealed class UnmanagedDriver : Name<bool>
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultCompletedJobHandler.cs
similarity index 67%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultCompletedJobHandler.cs
index 29a7d8e..1dc5d3c 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultCompletedJobHandler.cs
@@ -15,26 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Client;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Default
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DefaultCompletedJobHandler : IObserver<ICompletedJob>
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DefaultCompletedJobHandler));
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DefaultCompletedJobHandler()
         {
         }
 
-        public void OnNext(IFailedTask value)
+        public void OnCompleted()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            throw new NotImplementedException();
         }
 
         public void OnError(Exception error)
@@ -42,9 +41,9 @@ namespace Org.Apache.REEF.Driver.Defaults
             throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public void OnNext(ICompletedJob value)
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Info, "Job {0} completed", value.Id);
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultFailedJobHandler.cs
similarity index 67%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultFailedJobHandler.cs
index 29a7d8e..db71898 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultFailedJobHandler.cs
@@ -15,26 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Client;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Default
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DefaultFailedJobHandler : IObserver<IFailedJob>
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DefaultFailedJobHandler));
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DefaultFailedJobHandler()
         {
         }
 
-        public void OnNext(IFailedTask value)
+        public void OnCompleted()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            throw new NotImplementedException();
         }
 
         public void OnError(Exception error)
@@ -42,9 +41,9 @@ namespace Org.Apache.REEF.Driver.Defaults
             throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public void OnNext(IFailedJob value)
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Error, $"Job {value.Id} failed", value.AsError());
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultFailedRuntimeHandler.cs
similarity index 67%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultFailedRuntimeHandler.cs
index 29a7d8e..f4ccfd3 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultFailedRuntimeHandler.cs
@@ -15,26 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Client;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Default
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DefaultFailedRuntimeHandler : IObserver<IFailedRuntime>
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DefaultFailedRuntimeHandler));
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DefaultFailedRuntimeHandler()
         {
         }
 
-        public void OnNext(IFailedTask value)
+        public void OnCompleted()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            throw new NotImplementedException();
         }
 
         public void OnError(Exception error)
@@ -42,9 +41,9 @@ namespace Org.Apache.REEF.Driver.Defaults
             throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public void OnNext(IFailedRuntime value)
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Error, $"Job {value.Id} runtime failure", value.AsError());
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultJobMessageHandler.cs
similarity index 55%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultJobMessageHandler.cs
index 29a7d8e..19d80e9 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultJobMessageHandler.cs
@@ -15,26 +15,27 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Client;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Runtime.Serialization;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Default
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DefaultJobMessageHandler : IObserver<IJobMessage>
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DefaultJobMessageHandler));
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DefaultJobMessageHandler()
         {
         }
 
-        public void OnNext(IFailedTask value)
+        public void OnCompleted()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            throw new NotImplementedException();
         }
 
         public void OnError(Exception error)
@@ -42,9 +43,19 @@ namespace Org.Apache.REEF.Driver.Defaults
             throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public void OnNext(IJobMessage value)
         {
-            throw new NotImplementedException();
+            var message = "unable to deserialize message";
+            try
+            {
+                message = ByteUtilities.ByteArraysToString(value.Message);
+            }
+            catch (SerializationException)
+            {
+                var opaque = ByteUtilities.DeserializeFromBinaryFormat(value.Message);
+                message = opaque.ToString();
+            }
+            Log.Log(Level.Info, "Job {0} sent message {1}", value.Id, message);
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultRunningJobHandler.cs
similarity index 67%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultRunningJobHandler.cs
index 29a7d8e..2e35da6 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultRunningJobHandler.cs
@@ -15,26 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Client;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Default
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DefaultRunningJobHandler : IObserver<IRunningJob>
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DefaultRunningJobHandler));
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DefaultRunningJobHandler()
         {
         }
 
-        public void OnNext(IFailedTask value)
+        public void OnCompleted()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            throw new NotImplementedException();
         }
 
         public void OnError(Exception error)
@@ -42,9 +41,9 @@ namespace Org.Apache.REEF.Driver.Defaults
             throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public void OnNext(IRunningJob value)
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Info, "Job {0} is running", value.Id);
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultSubmittedJobHandler.cs
similarity index 67%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultSubmittedJobHandler.cs
index 29a7d8e..4a919e4 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultSubmittedJobHandler.cs
@@ -15,26 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Client;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Default
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DefaultSubmittedJobHandler : IObserver<ISubmittedJob>
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DefaultSubmittedJobHandler));
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DefaultSubmittedJobHandler()
         {
         }
 
-        public void OnNext(IFailedTask value)
+        public void OnCompleted()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            throw new NotImplementedException();
         }
 
         public void OnError(Exception error)
@@ -42,9 +41,9 @@ namespace Org.Apache.REEF.Driver.Defaults
             throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public void OnNext(ISubmittedJob value)
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Info, "Job {0} submitted", value.Id);
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultWakeErrorHandler.cs
similarity index 67%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultWakeErrorHandler.cs
index 29a7d8e..95ba1fa 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Default/DefaultWakeErrorHandler.cs
@@ -15,26 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Client;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Default
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DefaultWakeErrorHandler : IObserver<IWakeError>
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DefaultWakeErrorHandler));
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DefaultWakeErrorHandler()
         {
         }
 
-        public void OnNext(IFailedTask value)
+        public void OnCompleted()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            throw new NotImplementedException();
         }
 
         public void OnError(Exception error)
@@ -42,9 +41,9 @@ namespace Org.Apache.REEF.Driver.Defaults
             throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public void OnNext(IWakeError value)
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Error, $"Job {value.Id} wake error", value.AsError());
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/CompletedJob.cs
similarity index 75%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/CompletedJob.cs
index 2f3352a..92b2a58 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/CompletedJob.cs
@@ -15,12 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Common.Client;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class CompletedJob : ICompletedJob
     {
+        public CompletedJob(string id)
+        {
+            Id = id;
+        }
+
+        public string Id { get; }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/FailedJob.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/FailedJob.cs
new file mode 100644
index 0000000..55214a2
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/FailedJob.cs
@@ -0,0 +1,64 @@
+// 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.
+
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Common.Exceptions;
+using Org.Apache.REEF.Utilities;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Events
+{
+    internal sealed class FailedJob : IFailedJob
+    {
+        public FailedJob(string jobId, string message, byte[] data = null)
+        {
+            Id = jobId;
+            Message = message;
+            Data = Optional<byte[]>.OfNullable(data);
+        }
+
+        public string Id { get; }
+        public string Message { get; set; }
+        public Optional<string> Description { get; set; }
+        public Optional<string> Reason { get; set; }
+        public Optional<byte[]> Data { get; set; }
+
+        public Exception AsError()
+        {
+            if (Data.IsPresent())
+            {
+                Exception inner;
+                try
+                {
+                    inner = (Exception)ByteUtilities.DeserializeFromBinaryFormat(Data.Value);
+                }
+                catch (SerializationException se)
+                {
+                    inner = NonSerializableEvaluatorException.UnableToDeserialize(
+                        "Not able to deserialize job exception.",
+                        se);
+                }
+                return new JobException(Id, Message, inner);
+            }
+            else
+            {
+                return new JobException(Id, Message);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/FailedRuntime.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/FailedRuntime.cs
new file mode 100644
index 0000000..3c09ce7
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/FailedRuntime.cs
@@ -0,0 +1,64 @@
+// 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.
+
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Common.Exceptions;
+using Org.Apache.REEF.Utilities;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Events
+{
+    internal sealed class FailedRuntime : IFailedRuntime
+    {
+        public string Id { get; }
+        public string Message { get; set; }
+        public Optional<string> Description { get; set; }
+        public Optional<string> Reason { get; set; }
+        public Optional<byte[]> Data { get; set; }
+
+        public FailedRuntime(string id, string message, byte[] data = null)
+        {
+            Id = id;
+            Message = message;
+            Data = Optional<byte[]>.OfNullable(data);
+        }
+
+        public Exception AsError()
+        {
+            if (Data.IsPresent())
+            {
+                Exception inner;
+                try
+                {
+                    inner = (Exception)ByteUtilities.DeserializeFromBinaryFormat(Data.Value);
+                }
+                catch (SerializationException se)
+                {
+                    inner = NonSerializableEvaluatorException.UnableToDeserialize(
+                        "Exception from Evaluator was not able to be deserialized, returning a NonSerializableEvaluatorException.",
+                        se);
+                }
+                return new JobException(Id, Message, inner);
+            }
+            else
+            {
+                return new JobException(Id, Message);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/JobMessage.cs
similarity index 70%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/JobMessage.cs
index 2f3352a..d922edb 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/JobMessage.cs
@@ -15,12 +15,19 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Common.Client;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class JobMessage : IJobMessage
     {
+        public JobMessage(string id, byte[] message)
+        {
+            Id = id;
+            Message = message;
+        }
+
+        public string Id { get; }
+        public byte[] Message { get; }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/RunningJob.cs
similarity index 54%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/RunningJob.cs
index 2f3352a..c4934a4 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/RunningJob.cs
@@ -15,12 +15,35 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Common.Client;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class RunningJob : IRunningJob
     {
+        private readonly IClientService _clientService;
+
+        public RunningJob(IClientService clientService, string jobId)
+        {
+            _clientService = clientService;
+            Id = jobId;
+        }
+
+        public string Id { get; }
+
+        public void Dispose()
+        {
+            _clientService.Close();
+        }
+
+        public void Dispose(byte[] message)
+        {
+            _clientService.Close(message);
+        }
+
+        public void Send(byte[] message)
+        {
+            _clientService.Send(message);
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/SubmittedJob.cs
similarity index 75%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/SubmittedJob.cs
index 2f3352a..36c106b 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/SubmittedJob.cs
@@ -15,12 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Common.Client;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class SubmittedJob : ISubmittedJob
     {
+        public string Id { get; }
+
+        public SubmittedJob(string jobId)
+        {
+            Id = jobId;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/WakeError.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/WakeError.cs
new file mode 100644
index 0000000..0ba47d9
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/Events/WakeError.cs
@@ -0,0 +1,68 @@
+// 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.
+
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Common.Exceptions;
+using Org.Apache.REEF.Utilities;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Client.Events
+{
+    internal sealed class WakeError : IWakeError
+    {
+        public WakeError(string id, string message) : this(id, message, Optional<byte[]>.Empty())
+        {
+        }
+
+        public WakeError(string id, string message, Optional<byte[]> data)
+        {
+            Id = id;
+            Message = message;
+            Data = data;
+        }
+
+        public string Id { get; }
+        public string Message { get; set; }
+        public Optional<string> Description { get; set; }
+        public Optional<string> Reason { get; set; }
+        public Optional<byte[]> Data { get; set; }
+
+        public Exception AsError()
+        {
+            if (Data.IsPresent())
+            {
+                Exception inner;
+                try
+                {
+                    inner = (Exception)ByteUtilities.DeserializeFromBinaryFormat(Data.Value);
+                }
+                catch (SerializationException se)
+                {
+                    inner = NonSerializableEvaluatorException.UnableToDeserialize(
+                        "Exception from Evaluator was not able to be deserialized, returning a NonSerializableEvaluatorException.",
+                        se);
+                }
+                return new JobException(Id, Message, inner);
+            }
+            else
+            {
+                return new JobException(Id, Message);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/IClientLauncher.cs
similarity index 56%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/IClientLauncher.cs
index 2f3352a..ff2b7f2 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/IClientLauncher.cs
@@ -15,12 +15,23 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Tang.Interface;
+using System;
+using System.Threading;
+using System.Threading.Tasks;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <inheritdoc />
+    /// <summary>
+    /// All client launchers implement this interface. Multiple jobs can be submitted until the
+    /// launcher has been disposed <see cref="T:System.IDisposable" />
+    /// </summary>
+    public interface IClientLauncher : IDisposable
     {
+        Task<LauncherStatus> SubmitAsync(
+            IConfiguration driverAppConfiguration,
+            CancellationToken cancellationToken = default);
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/IClientService.cs
similarity index 56%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/IClientService.cs
index 2f3352a..ecaa80d 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/IClientService.cs
@@ -15,12 +15,24 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Client
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <summary>
+    /// Interface that all client services implement that provide
+    /// control flow channels to the driver.
+    /// </summary>
+    internal interface IClientService
     {
+        /// <summary>
+        /// Shutdown the job with an optional message.
+        /// </summary>
+        /// <param name="message">to deliver to the driver</param>
+        void Close(byte[] message = null);
+
+        /// <summary>
+        /// Send the driver a message.
+        /// </summary>
+        /// <param name="message">to deliver to the driver</param>
+        void Send(byte[] message);
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/JavaClientLauncher.cs
similarity index 60%
copy from lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/JavaClientLauncher.cs
index bc90c44..1c75d74 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Client/JavaClientLauncher.cs
@@ -15,31 +15,31 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using Org.Apache.REEF.Utilities.Logging;
 using System;
 using System.Collections.Generic;
 using System.Diagnostics;
 using System.Globalization;
 using System.IO;
 using System.Linq;
+using System.Threading;
 using System.Threading.Tasks;
-using Org.Apache.REEF.Client.API.Exceptions;
-using Org.Apache.REEF.Tang.Annotations;
-using Org.Apache.REEF.Utilities.Logging;
 
-namespace Org.Apache.REEF.Client.Common
+namespace Org.Apache.REEF.Bridge.Core.Common.Client
 {
     /// <summary>
     /// Helper class to launch the java side of the various clients.
     /// </summary>
-    internal class JavaClientLauncher : IJavaClientLauncher
+    internal sealed class JavaClientLauncher
     {
         /// <summary>
         /// The folder in which we search for the client jar.
         /// </summary>
         private const string JarFolder = "./";
 
-        private static readonly Logger Logger = Logger.GetLogger(typeof(JavaClientLauncher));
-        private readonly IList<string> _additionalClasspathEntries = new List<string>();
+        private static readonly Logger Log = Logger.GetLogger(typeof(JavaClientLauncher));
 
         [Inject]
         private JavaClientLauncher()
@@ -49,10 +49,15 @@ namespace Org.Apache.REEF.Client.Common
         /// <summary>
         /// Launch a java class in ClientConstants.ClientJarFilePrefix with provided parameters.
         /// </summary>
-        /// <param name="javaLogLevel"></param>
-        /// <param name="javaClassName"></param>
-        /// <param name="parameters"></param>
-        public Task LaunchAsync(JavaLoggingSetting javaLogLevel, string javaClassName, params string[] parameters)
+        /// <param name="javaLogLevel">Java logging level</param>
+        /// <param name="javaClassName">Java class that launches the Java driver</param>
+        /// <param name="parameters">Parameters associated with the Java main class</param>
+        /// <param name="cancellationToken">Token to cancel the launch</param>
+        public Task LaunchAsync(
+            JavaLoggingSetting javaLogLevel,
+            string javaClassName,
+            string[] parameters,
+            CancellationToken cancellationToken = default)
         {
             var startInfo = new ProcessStartInfo
             {
@@ -62,39 +67,41 @@ namespace Org.Apache.REEF.Client.Common
                 RedirectStandardOutput = true,
                 RedirectStandardError = true,
             };
-
-            var msg = string.Format(CultureInfo.CurrentCulture, "Launch Java with command: {0} {1}",
-                startInfo.FileName, startInfo.Arguments);
-            Logger.Log(Level.Info, msg);
-
-            var process = Process.Start(startInfo);
+            Log.Log(Level.Info, "Launch Java with command: {0} {1}", startInfo.FileName, startInfo.Arguments);
             var processExitTracker = new TaskCompletionSource<bool>();
-            if (process != null)
+            var process = new Process
             {
-                process.EnableRaisingEvents = true;
-                process.OutputDataReceived += delegate(object sender, DataReceivedEventArgs e)
+                StartInfo = startInfo,
+                EnableRaisingEvents = true
+            };
+            process.Exited += (sender, args) => { processExitTracker.SetResult(process.ExitCode == 0); };
+            if (cancellationToken != default)
+            {
+                cancellationToken.Register(processExitTracker.SetCanceled);
+            }
+            process.OutputDataReceived += delegate (object sender, DataReceivedEventArgs e)
+            {
+                if (!string.IsNullOrWhiteSpace(e.Data))
                 {
-                    if (!string.IsNullOrWhiteSpace(e.Data))
-                    {
-                        Logger.Log(Level.Info, e.Data);
-                    }
-                };
-                process.ErrorDataReceived += delegate(object sender, DataReceivedEventArgs e)
+                    Log.Log(Level.Info, e.Data);
+                }
+            };
+            process.ErrorDataReceived += delegate (object sender, DataReceivedEventArgs e)
+            {
+                if (!string.IsNullOrWhiteSpace(e.Data))
                 {
-                    if (!string.IsNullOrWhiteSpace(e.Data))
-                    {
-                        Logger.Log(Level.Error, e.Data);
-                    }
-                };
-                process.BeginErrorReadLine();
-                process.BeginOutputReadLine();
-                process.Exited += (sender, args) => { processExitTracker.SetResult(process.ExitCode == 0); };
+                    Log.Log(Level.Error, e.Data);
+                }
+            };
+            if (!process.Start())
+            {
+                processExitTracker.SetException(new Exception("Java client process didn't start."));
             }
             else
             {
-                processExitTracker.SetException(new Exception("Java client process didn't start."));
+                process.BeginErrorReadLine();
+                process.BeginOutputReadLine();
             }
-
             return processExitTracker.Task;
         }
 
@@ -117,11 +124,7 @@ namespace Org.Apache.REEF.Client.Common
             arguments.Add("-cp");
             arguments.Add(GetClientClasspath());
             arguments.Add(javaClassName);
-            foreach (var parameter in parameters)
-            {
-                arguments.Add(parameter);
-            }
-            return string.Join(" ", arguments);
+            return string.Join(" ", arguments.Concat(parameters));
         }
 
         /// <summary>
@@ -135,27 +138,28 @@ namespace Org.Apache.REEF.Client.Common
             if (string.IsNullOrWhiteSpace(javaHomePath))
             {
                 // TODO: Attempt to find java via the registry.
-                throw new JavaNotFoundException("JAVA_HOME isn't set. Please install Java and make set JAVA_HOME");
+                throw new IllegalStateException("JAVA_HOME isn't set. Please install Java and make set JAVA_HOME");
             }
 
             if (!Directory.Exists(javaHomePath))
             {
-                throw new JavaNotFoundException("JAVA_HOME references a folder that doesn't exist.", javaHomePath);
+                throw new IllegalStateException($"JAVA_HOME references a folder that doesn\'t exist. {javaHomePath}");
             }
 
             var javaBinPath = Path.Combine(javaHomePath, "bin");
             if (!Directory.Exists(javaBinPath))
             {
-                throw new JavaNotFoundException(
-                    "JAVA_HOME references a folder that doesn't contain a `bin` folder. Please adjust JAVA_HOME",
-                    javaHomePath);
+                throw new IllegalStateException(
+                    $"JAVA_HOME references a folder that doesn\'t contain a `bin` folder. Please adjust JAVA_HOME {javaHomePath}");
             }
 
             var javaPath = Path.Combine(javaBinPath, "java.exe");
+            if (File.Exists(javaPath)) return javaPath;
+            javaPath = Path.Combine(javaBinPath, "java");
             if (!File.Exists(javaPath))
             {
-                throw new JavaNotFoundException(
-                    "Could not find java.exe on this machine. Is Java installed and JAVA_HOME set?", javaPath);
+                throw new IllegalStateException(
+                    "Could not find java executable on this machine. Is Java installed and JAVA_HOME set? " + javaBinPath);
             }
             return javaPath;
         }
@@ -165,33 +169,17 @@ namespace Org.Apache.REEF.Client.Common
         /// </summary>
         /// <exception cref="ClasspathException">If the classpath would be empty.</exception>
         /// <returns></returns>
-        private string GetClientClasspath()
+        private static string GetClientClasspath()
         {
             var files = Directory.GetFiles(JarFolder)
-                .Where(x => (!string.IsNullOrWhiteSpace(x)))
-                .Where(e => Path.GetFileName(e).ToLower().StartsWith(ClientConstants.ClientJarFilePrefix))
+                .Where(e => Path.GetFileName(e).ToLower().Contains("reef-bridge-proto-java"))
                 .ToList();
-
             if (files.Count == 0)
             {
-                throw new ClasspathException(
-                    "Unable to assemble classpath. Make sure the REEF JAR is in the current working directory.");
-            }
-
-            var classpathEntries = new List<string>(_additionalClasspathEntries).Concat(files);
-            return string.Join(";", classpathEntries);
-        }
-
-        /// <summary>
-        /// Add entries to the end of the classpath of the java client.
-        /// </summary>
-        /// <param name="entries"></param>
-        public void AddToClassPath(IEnumerable<string> entries)
-        {
-            foreach (var entry in entries)
-            {
-                _additionalClasspathEntries.Add(entry);
+                throw new IllegalStateException(
+                    "Unable to assemble classpath. Make sure the REEF Jar is in the current working directory.");
             }
+            return string.Join(";", files);
         }
     }
 }
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/BridgeClock.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/BridgeClock.cs
new file mode 100644
index 0000000..227b186
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/BridgeClock.cs
@@ -0,0 +1,119 @@
+// 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.
+
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Implementations.InjectionPlan;
+using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Wake.Time;
+using Org.Apache.REEF.Wake.Time.Event;
+using Org.Apache.REEF.Wake.Time.Runtime.Event;
+using System;
+using System.Collections.Generic;
+using System.Threading;
+using System.Threading.Tasks;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver
+{
+    internal sealed class BridgeClock : IClock
+    {
+        internal struct AlarmInfo
+        {
+            public string AlarmId { get; set; }
+
+            public long Timestamp { get; set; }
+        }
+
+        private static readonly Logger Log = Logger.GetLogger(typeof(BridgeClock));
+
+        private readonly IInjectionFuture<IDriverServiceClient> _driverServiceClient;
+
+        private readonly IInjectionFuture<IDriverClientService> _driverClientService;
+
+        private readonly IDictionary<string, IObserver<Alarm>> _alarmDictionary = new Dictionary<string, IObserver<Alarm>>();
+
+        private int _outstandingAlarms;
+
+        [Inject]
+        private BridgeClock(
+            IInjectionFuture<IDriverServiceClient> driverServiceClient,
+            IInjectionFuture<IDriverClientService> driverClientService)
+        {
+            _driverServiceClient = driverServiceClient;
+            _driverClientService = driverClientService;
+            _outstandingAlarms = 0;
+        }
+
+        public void Dispose()
+        {
+            _driverServiceClient.Get().OnShutdown();
+        }
+
+        public void Dispose(Exception ex)
+        {
+            _driverServiceClient.Get().OnShutdown(ex);
+        }
+
+        public bool IsIdle()
+        {
+            return _outstandingAlarms == 0 && _alarmDictionary.Count == 0;
+        }
+
+        public void ScheduleAlarm(long offset, IObserver<Alarm> handler)
+        {
+            var alarmId = Guid.NewGuid().ToString();
+            _alarmDictionary[alarmId] = handler;
+            Log.Log(Level.Info, "Schedule alarm id {0} for {1}", alarmId, offset);
+            _driverServiceClient.Get().OnSetAlarm(alarmId, offset);
+        }
+
+        public void Run()
+        {
+            _driverClientService.Get().Start();
+            _driverClientService.Get().AwaitTermination();
+            Log.Log(Level.Info, "bridge clock terminated");
+        }
+
+        public void OnNext(AlarmInfo alarmInfo)
+        {
+            OnNextAsync(alarmInfo).GetAwaiter().GetResult();
+        }
+
+        public async Task OnNextAsync(AlarmInfo alarmInfo)
+        {
+            Log.Log(Level.Info, "scheduling alarm id {0}", alarmInfo.AlarmId);
+            if (_alarmDictionary.TryGetValue(alarmInfo.AlarmId, out IObserver<Alarm> alarmObserver))
+            {
+                Interlocked.Increment(ref _outstandingAlarms);
+                try
+                {
+                    var alarm = new RuntimeAlarm(alarmInfo.Timestamp, alarmObserver);
+                    _alarmDictionary.Remove(alarmInfo.AlarmId);
+                    await Task.Run(() => alarm.Handle());
+                }
+                catch (Exception ex)
+                {
+                    Log.Log(Level.Error, "Alarm handler raised exception", ex);
+                    Dispose(ex);
+                }
+                finally
+                {
+                    Interlocked.Decrement(ref _outstandingAlarms);
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DispatchEventHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DispatchEventHandler.cs
new file mode 100644
index 0000000..a7f9367
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DispatchEventHandler.cs
@@ -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.
+
+using System;
+using System.Collections.Generic;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver
+{
+    internal sealed class DispatchEventHandler<T> : IObserver<T>, IObservable<T>
+    {
+        private readonly ISet<IObserver<T>> _userHandlers;
+
+        public DispatchEventHandler()
+        {
+            _userHandlers = new HashSet<IObserver<T>>();
+        }
+
+        public DispatchEventHandler(ISet<IObserver<T>> userHandlers)
+        {
+            _userHandlers = userHandlers;
+        }
+
+        public void OnNext(T value)
+        {
+            foreach (var observer in _userHandlers)
+            {
+                observer.OnNext(value);
+            }
+        }
+
+        public void OnError(Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+
+        public IDisposable Subscribe(IObserver<T> observer)
+        {
+            _userHandlers.Add(observer);
+            return new DisposeImpl()
+            {
+                DispatchEventHandler = this,
+                Handler = observer
+            };
+        }
+
+        private class DisposeImpl : IDisposable
+        {
+            public DispatchEventHandler<T> DispatchEventHandler { private get; set; }
+            public IObserver<T> Handler { private get; set; }
+
+            public void Dispose()
+            {
+                DispatchEventHandler._userHandlers.Remove(Handler);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridge.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridge.cs
new file mode 100644
index 0000000..d6db5c8
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridge.cs
@@ -0,0 +1,353 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config;
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Globalization;
+using System.Threading;
+using System.Threading.Tasks;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver
+{
+    /// <summary>
+    /// DriverBridge is responsible for running application handlers and keeping
+    /// track of how many are currently active. It exposes a method <see cref="IsIdle"/>
+    /// that indicates if there are any active handlers, which is used to determine
+    /// (among other things) whether the driver is currently idle.
+    /// </summary>
+    internal sealed class DriverBridge
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DriverBridge));
+
+        // Control event dispatchers
+
+        private readonly DispatchEventHandler<IDriverStarted> _driverStartedDispatcher;
+
+        private readonly DispatchEventHandler<IDriverStopped> _driverStoppedDispatcher;
+
+        // Evaluator event dispatchers
+
+        private readonly DispatchEventHandler<IAllocatedEvaluator> _allocatedEvaluatorDispatcher;
+
+        private readonly DispatchEventHandler<IFailedEvaluator> _failedEvaluatorDispatcher;
+
+        private readonly DispatchEventHandler<ICompletedEvaluator> _completedEvaluatorDispatcher;
+
+        // Context event dispatchers
+
+        private readonly DispatchEventHandler<IActiveContext> _activeContextDispatcher;
+
+        private readonly DispatchEventHandler<IClosedContext> _closedContextDispatcher;
+
+        private readonly DispatchEventHandler<IFailedContext> _failedContextDispatcher;
+
+        private readonly DispatchEventHandler<IContextMessage> _contextMessageDispatcher;
+
+        // Task event dispatchers
+
+        private readonly DispatchEventHandler<ITaskMessage> _taskMessageDispatcher;
+
+        private readonly DispatchEventHandler<IFailedTask> _failedTaskDispatcher;
+
+        private readonly DispatchEventHandler<IRunningTask> _runningTaskDispatcher;
+
+        private readonly DispatchEventHandler<ICompletedTask> _completedTaskDispatcher;
+
+        private readonly DispatchEventHandler<ISuspendedTask> _suspendedTaskDispatcher;
+
+        // Driver restart event dispatchers
+
+        private readonly DispatchEventHandler<IDriverRestarted> _driverRestartedDispatcher;
+
+        private readonly DispatchEventHandler<IActiveContext> _driverRestartActiveContextDispatcher;
+
+        private readonly DispatchEventHandler<IRunningTask> _driverRestartRunningTaskDispatcher;
+
+        private readonly DispatchEventHandler<IDriverRestartCompleted> _driverRestartCompletedDispatcher;
+
+        private readonly DispatchEventHandler<IFailedEvaluator> _driverRestartFailedEvaluatorDispatcher;
+
+        // Client event handlers
+
+        private readonly DispatchEventHandler<byte[]> _clientCloseDispatcher;
+
+        private readonly DispatchEventHandler<byte[]> _clientCloseWithMessageDispatcher;
+
+        private readonly DispatchEventHandler<byte[]> _clientMessageDispatcher;
+
+        private static int s_activeDispatchCounter;
+
+        public static bool IsIdle => s_activeDispatchCounter == 0;
+
+        [Inject]
+        private DriverBridge(
+            // Runtime events
+            [Parameter(Value = typeof(DriverApplicationParameters.DriverStartedHandlers))]
+            ISet<IObserver<IDriverStarted>> driverStartHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.DriverStopHandlers))]
+            ISet<IObserver<IDriverStopped>> driverStopHandlers,
+            // Evaluator events
+            [Parameter(Value = typeof(DriverApplicationParameters.AllocatedEvaluatorHandlers))]
+            ISet<IObserver<IAllocatedEvaluator>> allocatedEvaluatorHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.FailedEvaluatorHandlers))]
+            ISet<IObserver<IFailedEvaluator>> failedEvaluatorHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.CompletedEvaluatorHandlers))]
+            ISet<IObserver<ICompletedEvaluator>> completedEvaluatorHandlers,
+            // Context events
+            [Parameter(Value = typeof(DriverApplicationParameters.ActiveContextHandlers))]
+            ISet<IObserver<IActiveContext>> activeContextHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.ClosedContextHandlers))]
+            ISet<IObserver<IClosedContext>> closedContextHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.FailedContextHandlers))]
+            ISet<IObserver<IFailedContext>> failedContextHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.ContextMessageHandlers))]
+            ISet<IObserver<IContextMessage>> contextMessageHandlers,
+            // Task events
+            [Parameter(Value = typeof(DriverApplicationParameters.TaskMessageHandlers))]
+            ISet<IObserver<ITaskMessage>> taskMessageHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.FailedTaskHandlers))]
+            ISet<IObserver<IFailedTask>> failedTaskHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.RunningTaskHandlers))]
+            ISet<IObserver<IRunningTask>> runningTaskHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.CompletedTaskHandlers))]
+            ISet<IObserver<ICompletedTask>> completedTaskHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.SuspendedTaskHandlers))]
+            ISet<IObserver<ISuspendedTask>> suspendedTaskHandlers,
+            // Driver restart events
+            [Parameter(Value = typeof(DriverApplicationParameters.DriverRestartedHandlers))]
+            ISet<IObserver<IDriverRestarted>> driverRestartedHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.DriverRestartActiveContextHandlers))]
+            ISet<IObserver<IActiveContext>> driverRestartActiveContextHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.DriverRestartRunningTaskHandlers))]
+            ISet<IObserver<IRunningTask>> driverRestartRunningTaskHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.DriverRestartCompletedHandlers))]
+            ISet<IObserver<IDriverRestartCompleted>> driverRestartCompletedHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.DriverRestartFailedEvaluatorHandlers))]
+            ISet<IObserver<IFailedEvaluator>> driverRestartFailedEvaluatorHandlers,
+            // Client event
+            [Parameter(Value = typeof(DriverApplicationParameters.ClientCloseWithMessageHandlers))]
+            ISet<IObserver<byte[]>> clientCloseWithMessageHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.ClientCloseHandlers))]
+            ISet<IObserver<byte[]>> clientCloseHandlers,
+            [Parameter(Value = typeof(DriverApplicationParameters.ClientMessageHandlers))]
+            ISet<IObserver<byte[]>> clientMessageHandlers,
+            // Misc.
+            [Parameter(Value = typeof(DriverApplicationParameters.TraceListeners))]
+            ISet<TraceListener> traceListeners,
+            [Parameter(Value = typeof(DriverApplicationParameters.TraceLevel))]
+            string traceLevel)
+        {
+            _driverStartedDispatcher = new DispatchEventHandler<IDriverStarted>(driverStartHandlers);
+            _driverStoppedDispatcher = new DispatchEventHandler<IDriverStopped>(driverStopHandlers);
+            _allocatedEvaluatorDispatcher = new DispatchEventHandler<IAllocatedEvaluator>(allocatedEvaluatorHandlers);
+            _failedEvaluatorDispatcher = new DispatchEventHandler<IFailedEvaluator>(failedEvaluatorHandlers);
+            _completedEvaluatorDispatcher = new DispatchEventHandler<ICompletedEvaluator>(completedEvaluatorHandlers);
+            _activeContextDispatcher = new DispatchEventHandler<IActiveContext>(activeContextHandlers);
+            _closedContextDispatcher = new DispatchEventHandler<IClosedContext>(closedContextHandlers);
+            _failedContextDispatcher = new DispatchEventHandler<IFailedContext>(failedContextHandlers);
+            _contextMessageDispatcher = new DispatchEventHandler<IContextMessage>(contextMessageHandlers);
+            _taskMessageDispatcher = new DispatchEventHandler<ITaskMessage>(taskMessageHandlers);
+            _failedTaskDispatcher = new DispatchEventHandler<IFailedTask>(failedTaskHandlers);
+            _runningTaskDispatcher = new DispatchEventHandler<IRunningTask>(runningTaskHandlers);
+            _completedTaskDispatcher = new DispatchEventHandler<ICompletedTask>(completedTaskHandlers);
+            _suspendedTaskDispatcher = new DispatchEventHandler<ISuspendedTask>(suspendedTaskHandlers);
+            _driverRestartedDispatcher = new DispatchEventHandler<IDriverRestarted>(driverRestartedHandlers);
+            _driverRestartActiveContextDispatcher = new DispatchEventHandler<IActiveContext>(driverRestartActiveContextHandlers);
+            _driverRestartRunningTaskDispatcher = new DispatchEventHandler<IRunningTask>(driverRestartRunningTaskHandlers);
+            _driverRestartCompletedDispatcher = new DispatchEventHandler<IDriverRestartCompleted>(driverRestartCompletedHandlers);
+            _driverRestartFailedEvaluatorDispatcher = new DispatchEventHandler<IFailedEvaluator>(driverRestartFailedEvaluatorHandlers);
+            _clientCloseDispatcher = new DispatchEventHandler<byte[]>(clientCloseHandlers);
+            _clientCloseWithMessageDispatcher = new DispatchEventHandler<byte[]>(clientCloseWithMessageHandlers);
+            _clientMessageDispatcher = new DispatchEventHandler<byte[]>(clientMessageHandlers);
+
+            foreach (var listener in traceListeners)
+            {
+                Logger.AddTraceListener(listener);
+            }
+            Log.Log(Level.Info, "Constructing DriverBridge");
+
+            if (Enum.TryParse(traceLevel.ToString(CultureInfo.InvariantCulture), out Level level))
+            {
+                Logger.SetCustomLevel(level);
+            }
+            else
+            {
+                Log.Log(Level.Warning, "Invalid trace level {0} provided, will by default use verbose level", traceLevel);
+            }
+            s_activeDispatchCounter = 0;
+        }
+
+        public async Task DispatchDriverRestartFailedEvaluatorEvent(IFailedEvaluator failedEvaluatorEvent)
+        {
+            await DispatchAsync(_driverRestartFailedEvaluatorDispatcher, failedEvaluatorEvent);
+        }
+
+        public async Task DispatchDriverRestartCompletedEvent(IDriverRestartCompleted driverRestartCompletedEvent)
+        {
+            await DispatchAsync(_driverRestartCompletedDispatcher, driverRestartCompletedEvent);
+        }
+
+        public async Task DispatchDriverRestartRunningTaskEvent(IRunningTask runningTaskEvent)
+        {
+            await DispatchAsync(_driverRestartRunningTaskDispatcher, runningTaskEvent);
+        }
+
+        public async Task DispatchDriverRestartActiveContextEvent(IActiveContext activeContextEvent)
+        {
+            await DispatchAsync(_driverRestartActiveContextDispatcher, activeContextEvent);
+        }
+
+        public async Task DispatchDriverRestartedEvent(IDriverRestarted driverRestartedEvent)
+        {
+            await DispatchAsync(_driverRestartedDispatcher, driverRestartedEvent);
+        }
+
+        public async Task DispatchCompletedTaskEvent(ICompletedTask completedTaskEvent)
+        {
+            await DispatchAsync(_completedTaskDispatcher, completedTaskEvent);
+        }
+
+        public async Task DispatchRunningTaskEvent(IRunningTask runningTaskEvent)
+        {
+            await DispatchAsync(_runningTaskDispatcher, runningTaskEvent);
+        }
+
+        public async Task DispatchFailedTaskEvent(IFailedTask failedTaskEvent)
+        {
+            await DispatchAsync(_failedTaskDispatcher, failedTaskEvent);
+        }
+
+        public async Task DispatchTaskMessageEvent(ITaskMessage taskMessageEvent)
+        {
+            await DispatchAsync(_taskMessageDispatcher, taskMessageEvent);
+        }
+
+        public async Task DispatchSuspendedTaskEvent(ISuspendedTask suspendedTask)
+        {
+            await DispatchAsync(_suspendedTaskDispatcher, suspendedTask);
+        }
+
+        public async Task DispatchContextMessageEvent(IContextMessage contextMessageEvent)
+        {
+            await DispatchAsync(_contextMessageDispatcher, contextMessageEvent);
+        }
+
+        public async Task DispatchFailedContextEvent(IFailedContext failedContextEvent)
+        {
+            await DispatchAsync(_failedContextDispatcher, failedContextEvent);
+        }
+
+        public async Task DispatchClosedContextEvent(IClosedContext closedContextEvent)
+        {
+            await DispatchAsync(_closedContextDispatcher, closedContextEvent);
+        }
+
+        public async Task DispatchActiveContextEvent(IActiveContext activeContextEvent)
+        {
+            await DispatchAsync(_activeContextDispatcher, activeContextEvent);
+        }
+
+        public async Task DispatchCompletedEvaluatorEvent(ICompletedEvaluator completedEvaluatorEvent)
+        {
+            await DispatchAsync(_completedEvaluatorDispatcher, completedEvaluatorEvent);
+        }
+
+        public async Task DispatchFailedEvaluatorEvent(IFailedEvaluator failedEvaluatorEvent)
+        {
+            await DispatchAsync(_failedEvaluatorDispatcher, failedEvaluatorEvent);
+        }
+
+        public async Task DispatchAllocatedEvaluatorEventAsync(IAllocatedEvaluator allocatedEvaluatorEvent)
+        {
+            await DispatchAsync(_allocatedEvaluatorDispatcher, allocatedEvaluatorEvent);
+        }
+
+        public async Task DispatchStartEventAsync(IDriverStarted startEvent)
+        {
+            await DispatchAsync(_driverStartedDispatcher, startEvent);
+        }
+
+        public async Task DispatchStopEvent(IDriverStopped stopEvent)
+        {
+            await DispatchAsync(_driverStoppedDispatcher, stopEvent);
+        }
+
+        public async Task DispatchClientCloseEvent()
+        {
+            await DispatchAsync(_clientCloseDispatcher, null);
+        }
+
+        public async Task DispatchClientCloseWithMessageEvent(byte[] message)
+        {
+            await DispatchAsync(_clientCloseWithMessageDispatcher, message);
+        }
+
+        public async Task DispatchClientMessageEvent(byte[] message)
+        {
+            await DispatchAsync(_clientMessageDispatcher, message);
+        }
+
+        private static async Task DispatchAsync<T>(DispatchEventHandler<T> handler, T message)
+        {
+            using (var operation = new DisposableOperation(() => handler.OnNext(message)))
+            {
+                await operation.Run();
+            }
+        }
+
+        private sealed class DisposableOperation : IDisposable
+        {
+            private readonly Action _operation;
+
+            public DisposableOperation(Action operation)
+            {
+                _operation = operation;
+            }
+
+            public async Task Run()
+            {
+                try
+                {
+                    Interlocked.Increment(ref s_activeDispatchCounter);
+                    await Task.Run(_operation);
+                }
+                catch (Exception ex)
+                {
+                    Log.Log(Level.Error, "Operation error", ex);
+                    throw;
+                }
+                finally
+                {
+                    Interlocked.Decrement(ref s_activeDispatchCounter);
+                }
+            }
+
+            public void Dispose()
+            {
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridgeConfiguration.cs
similarity index 51%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridgeConfiguration.cs
index 2f3352a..340d1f1 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridgeConfiguration.cs
@@ -15,12 +15,20 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class DriverBridgeConfiguration : ConfigurationModuleBuilder
     {
+        public static readonly RequiredImpl<IDriverClientService> DriverClientService = new RequiredImpl<IDriverClientService>();
+
+        public static readonly RequiredImpl<IDriverServiceClient> DriverServiceClient = new RequiredImpl<IDriverServiceClient>();
+
+        public static ConfigurationModule ConfigurationModule => new DriverBridgeConfiguration()
+            .BindImplementation(GenericType<IDriverClientService>.Class, DriverClientService)
+            .BindImplementation(GenericType<IDriverServiceClient>.Class, DriverServiceClient)
+            .Build();
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridgeEvaluatorRequestor.cs
similarity index 53%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridgeEvaluatorRequestor.cs
index 29a7d8e..66a75a8 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/DriverBridgeEvaluatorRequestor.cs
@@ -15,36 +15,37 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Common.Catalog;
+using Org.Apache.REEF.Driver.Evaluator;
 using Org.Apache.REEF.Tang.Annotations;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class DriverBridgeEvaluatorRequestor : IEvaluatorRequestor
     {
+        private readonly IDriverServiceClient _driverServiceClient;
+
         [Inject]
-        public DefaultTaskFailureHandler()
+        private DriverBridgeEvaluatorRequestor(IDriverServiceClient driverServiceClient)
         {
+            _driverServiceClient = driverServiceClient;
         }
 
-        public void OnNext(IFailedTask value)
+        public IResourceCatalog ResourceCatalog { get; }
+
+        public void Submit(IEvaluatorRequest request)
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            _driverServiceClient.OnEvaluatorRequest(request);
         }
 
-        public void OnError(Exception error)
+        public EvaluatorRequestBuilder NewBuilder()
         {
-            throw new NotImplementedException();
+            return new EvaluatorRequestBuilder();
         }
 
-        public void OnCompleted()
+        public EvaluatorRequestBuilder NewBuilder(IEvaluatorRequest request)
         {
-            throw new NotImplementedException();
+            return new EvaluatorRequestBuilder(request);
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeActiveContext.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeActiveContext.cs
new file mode 100644
index 0000000..0bfc210
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeActiveContext.cs
@@ -0,0 +1,74 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities;
+using System;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class BridgeActiveContext : IActiveContext
+    {
+        private readonly IDriverServiceClient _driverServiceClient;
+
+        public string Id { get; }
+        public string EvaluatorId { get; }
+        public Optional<string> ParentId { get; }
+        public IEvaluatorDescriptor EvaluatorDescriptor { get; }
+
+        public BridgeActiveContext(
+            IDriverServiceClient driverServiceClient,
+            string id,
+            string evaluatorId,
+            Optional<string> parentId,
+            IEvaluatorDescriptor evaluatorDescriptor)
+        {
+            _driverServiceClient = driverServiceClient;
+            Id = id;
+            EvaluatorId = evaluatorId;
+            ParentId = parentId;
+            EvaluatorDescriptor = evaluatorDescriptor;
+        }
+
+        public void Dispose()
+        {
+            _driverServiceClient.OnContextClose(Id);
+        }
+
+        public void SubmitTask(IConfiguration taskConf)
+        {
+            _driverServiceClient.OnContextSubmitTask(Id, taskConf);
+        }
+
+        public void SubmitContext(IConfiguration contextConfiguration)
+        {
+            _driverServiceClient.OnContextSubmitContext(Id, contextConfiguration);
+        }
+
+        public void SubmitContextAndService(IConfiguration contextConfiguration, IConfiguration serviceConfiguration)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void SendMessage(byte[] message)
+        {
+            _driverServiceClient.OnContextMessage(Id, message);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeAllocatedEvaluator.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeAllocatedEvaluator.cs
new file mode 100644
index 0000000..9dcc918
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeAllocatedEvaluator.cs
@@ -0,0 +1,163 @@
+// 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.
+
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Common.Evaluator;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities;
+using System;
+using System.Collections.Generic;
+using System.IO;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class BridgeAllocatedEvaluator : IAllocatedEvaluator
+    {
+        private readonly IDriverServiceClient _driverServiceClient;
+
+        private readonly IEvaluatorDescriptor _evaluatorDescriptor;
+
+        private readonly List<FileInfo> _addFiles = new List<FileInfo>();
+
+        private readonly List<FileInfo> _addLibraries = new List<FileInfo>();
+
+        public string Id { get; }
+
+        public EvaluatorType Type
+        {
+            get { return EvaluatorType.CLR; }
+            set
+            {
+                throw new NotImplementedException();
+            }
+        }
+
+        public string NameServerInfo
+        {
+            get
+            {
+                throw new NotImplementedException();
+            }
+            set
+            {
+                throw new NotImplementedException();
+            }
+        }
+
+        public string EvaluatorBatchId
+        {
+            get
+            {
+                throw new NotImplementedException();
+            }
+            set
+            {
+                throw new NotImplementedException();
+            }
+        }
+
+        public BridgeAllocatedEvaluator(
+            string id,
+            IDriverServiceClient driverServiceClient,
+            IEvaluatorDescriptor evaluatorDescriptor)
+        {
+            Id = id;
+            _driverServiceClient = driverServiceClient;
+            _evaluatorDescriptor = evaluatorDescriptor;
+        }
+
+        public void Dispose()
+        {
+            _driverServiceClient.OnEvaluatorClose(Id);
+        }
+
+        public void SubmitContext(IConfiguration contextConfiguration)
+        {
+            _driverServiceClient.OnEvaluatorSubmit(
+                Id,
+                contextConfiguration,
+                Optional<IConfiguration>.Empty(),
+                Optional<IConfiguration>.Empty(),
+                _addFiles, _addLibraries);
+        }
+
+        public void SubmitContextAndService(IConfiguration contextConfiguration, IConfiguration serviceConfiguration)
+        {
+            _driverServiceClient.OnEvaluatorSubmit(
+                Id,
+                contextConfiguration,
+                Optional<IConfiguration>.Of(serviceConfiguration),
+                Optional<IConfiguration>.Empty(),
+                _addFiles, _addLibraries);
+        }
+
+        public void SubmitContextAndTask(IConfiguration contextConfiguration, IConfiguration taskConfiguration)
+        {
+            _driverServiceClient.OnEvaluatorSubmit(
+                Id,
+                contextConfiguration,
+                Optional<IConfiguration>.Empty(),
+                Optional<IConfiguration>.Of(taskConfiguration),
+                _addFiles, _addLibraries);
+        }
+
+        public void SubmitContextAndServiceAndTask(IConfiguration contextConfiguration, IConfiguration serviceConfiguration,
+            IConfiguration taskConfiguration)
+        {
+            _driverServiceClient.OnEvaluatorSubmit(
+                Id,
+                contextConfiguration,
+                Optional<IConfiguration>.Of(serviceConfiguration),
+                Optional<IConfiguration>.Of(taskConfiguration),
+                _addFiles, _addLibraries);
+        }
+
+        public void SubmitTask(IConfiguration taskConf)
+        {
+            var contextConfiguration =
+                ContextConfiguration.ConfigurationModule.Set(
+                    ContextConfiguration.Identifier, "RootContext_" + Id).Build();
+            _driverServiceClient.OnEvaluatorSubmit(
+                Id,
+                contextConfiguration,
+                Optional<IConfiguration>.Empty(),
+                Optional<IConfiguration>.Of(taskConf),
+                _addFiles, _addLibraries);
+        }
+
+        public IEvaluatorDescriptor GetEvaluatorDescriptor()
+        {
+            return _evaluatorDescriptor;
+        }
+
+        public void AddFile(string file)
+        {
+            _addFiles.Add(new FileInfo(file));
+        }
+
+        public void AddLibrary(string lib)
+        {
+            _addLibraries.Add(new FileInfo(lib));
+        }
+
+        public void AddFileResource(string file)
+        {
+            _addFiles.Add(new FileInfo(file));
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeClosedContext.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeClosedContext.cs
new file mode 100644
index 0000000..75c46c1
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeClosedContext.cs
@@ -0,0 +1,46 @@
+// 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.
+
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Utilities;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class BridgeClosedContext : IClosedContext
+    {
+        public string Id { get; }
+        public string EvaluatorId { get; }
+        public Optional<string> ParentId { get; }
+        public IEvaluatorDescriptor EvaluatorDescriptor { get; }
+        public IActiveContext ParentContext { get; }
+
+        public BridgeClosedContext(
+            string id,
+            string evaluatorId,
+            Optional<string> parentId,
+            IEvaluatorDescriptor evaluatorDescriptor,
+            IActiveContext parentContext)
+        {
+            Id = id;
+            EvaluatorId = evaluatorId;
+            ParentId = parentId;
+            EvaluatorDescriptor = evaluatorDescriptor;
+            ParentContext = parentContext;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeCompletedEvaluator.cs
similarity index 73%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeCompletedEvaluator.cs
index 2f3352a..99d94fa 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeCompletedEvaluator.cs
@@ -15,12 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver.Evaluator;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class BridgeCompletedEvaluator : ICompletedEvaluator
     {
+        public string Id { get; }
+
+        public BridgeCompletedEvaluator(string id)
+        {
+            Id = id;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeCompletedTask.cs
similarity index 60%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeCompletedTask.cs
index 2f3352a..087fafd 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeCompletedTask.cs
@@ -15,12 +15,22 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Task;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class BridgeCompletedTask : ICompletedTask
     {
+        public byte[] Message { get; }
+        public string Id { get; }
+        public IActiveContext ActiveContext { get; }
+
+        public BridgeCompletedTask(byte[] message, string id, IActiveContext activeContext)
+        {
+            Message = message;
+            Id = id;
+            ActiveContext = activeContext;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeContextMessage.cs
similarity index 62%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeContextMessage.cs
index 2f3352a..4e394af 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeContextMessage.cs
@@ -15,12 +15,21 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Common.Context;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class BridgeContextMessage : IContextMessage
     {
+        public byte[] Message { get; }
+        public string Id { get; }
+        public string MessageSourceId { get; }
+
+        public BridgeContextMessage(string id, string messageSourceId, byte[] message)
+        {
+            Message = message;
+            Id = id;
+            MessageSourceId = messageSourceId;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverRestartCompleted.cs
similarity index 64%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverRestartCompleted.cs
index 2f3352a..01db9e9 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverRestartCompleted.cs
@@ -15,12 +15,20 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver;
+using System;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class BridgeDriverRestartCompleted : IDriverRestartCompleted
     {
+        public DateTime CompletedTime { get; }
+        public bool IsTimedOut { get; }
+
+        public BridgeDriverRestartCompleted(DateTime completedTime, bool isTimedOut)
+        {
+            CompletedTime = completedTime;
+            IsTimedOut = isTimedOut;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverRestarted.cs
similarity index 56%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverRestarted.cs
index 2f3352a..ee73c65 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverRestarted.cs
@@ -15,12 +15,23 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver;
+using System;
+using System.Collections.Generic;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class BridgeDriverRestarted : IDriverRestarted
     {
+        public DateTime StartTime { get; }
+        public ISet<string> ExpectedEvaluatorIds { get; }
+        public int ResubmissionAttempts { get; }
+
+        public BridgeDriverRestarted(DateTime startTime, ISet<string> expectedEvaluatorIds, int resubmissionAttempts)
+        {
+            StartTime = startTime;
+            ExpectedEvaluatorIds = expectedEvaluatorIds;
+            ResubmissionAttempts = resubmissionAttempts;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverStarted.cs
similarity index 72%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverStarted.cs
index 2f3352a..4ac7f4e 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverStarted.cs
@@ -15,12 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver;
+using System;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal class BridgeDriverStarted : IDriverStarted
     {
+        public DateTime StartTime { get; }
+
+        public BridgeDriverStarted(DateTime startTime)
+        {
+            StartTime = startTime;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverStopped.cs
similarity index 72%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverStopped.cs
index 2f3352a..aa5eab7 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeDriverStopped.cs
@@ -15,12 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver;
+using System;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal class BridgeDriverStopped : IDriverStopped
     {
+        public DateTime StopTime { get; }
+
+        public BridgeDriverStopped(DateTime stopTime)
+        {
+            StopTime = stopTime;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedContext.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedContext.cs
new file mode 100644
index 0000000..1d190f2
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedContext.cs
@@ -0,0 +1,45 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Utilities;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class BridgeFailedContext : IFailedContext
+    {
+        public string Id { get; }
+        public string EvaluatorId { get; }
+        public Optional<string> ParentId { get; }
+        public IEvaluatorDescriptor EvaluatorDescriptor { get; }
+        public Optional<IActiveContext> ParentContext { get; }
+
+        public BridgeFailedContext(
+            string id,
+            string evaluatorId,
+            IEvaluatorDescriptor evaluatorDescriptor,
+            Optional<IActiveContext> parentContext)
+        {
+            Id = id;
+            EvaluatorId = evaluatorId;
+            ParentId = Optional<string>.OfNullable(parentContext.OrElse(null)?.Id);
+            EvaluatorDescriptor = evaluatorDescriptor;
+            ParentContext = parentContext;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedEvaluator.cs
similarity index 50%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedEvaluator.cs
index 29a7d8e..9bb24a4 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedEvaluator.cs
@@ -15,36 +15,31 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
-using System.Globalization;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
 using Org.Apache.REEF.Driver.Task;
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities;
+using System.Collections.Generic;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class BridgeFailedEvaluator : IFailedEvaluator
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
+        public BridgeFailedEvaluator(
+            string id,
+            EvaluatorException evaluatorException,
+            IList<IFailedContext> failedContexts,
+            Optional<IFailedTask> failedTask)
         {
+            Id = id;
+            EvaluatorException = evaluatorException;
+            FailedContexts = failedContexts;
+            FailedTask = failedTask;
         }
 
-        public void OnNext(IFailedTask value)
-        {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
-        }
-
-        public void OnError(Exception error)
-        {
-            throw new NotImplementedException();
-        }
-
-        public void OnCompleted()
-        {
-            throw new NotImplementedException();
-        }
+        public string Id { get; }
+        public EvaluatorException EvaluatorException { get; }
+        public IList<IFailedContext> FailedContexts { get; }
+        public Optional<IFailedTask> FailedTask { get; }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedTask.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedTask.cs
new file mode 100644
index 0000000..23b1c53f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeFailedTask.cs
@@ -0,0 +1,130 @@
+// 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.
+
+using Org.Apache.REEF.Common.Avro;
+using Org.Apache.REEF.Common.Exceptions;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class BridgeFailedTask : IFailedTask
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(BridgeFailedTask));
+
+        private readonly Optional<IActiveContext> _activeContext;
+
+        private Exception _cause = null;
+
+        public BridgeFailedTask(Optional<IActiveContext> activeContext, string id, string message, byte[] data) :
+            this(activeContext, id, message, null, null, data)
+        {
+        }
+
+        public BridgeFailedTask(
+            Optional<IActiveContext> activeContext,
+            string id,
+            string message,
+            string description,
+            string reason,
+            byte[] data)
+        {
+            _activeContext = activeContext;
+            Id = id;
+            Message = message;
+            Description = Optional<string>.OfNullable(description);
+            Reason = Optional<string>.OfNullable(reason);
+            if (data != null)
+            {
+                var avroFailedTask = AvroJsonSerializer<AvroFailedTask>.FromBytes(data);
+
+                if (!id.Equals(avroFailedTask.identifier))
+                {
+                    Log.Log(Level.Error, "Task error id {0} does not match task id {1}", avroFailedTask.identifier, id);
+                }
+
+                // Data is simply the serialized Exception.ToString.
+                Data = Optional<byte[]>.OfNullable(avroFailedTask.data);
+
+                // Message can be overwritten in Java, if the C# Message is null and the Task failure is caused by an Evaluator failure.
+                Message = string.IsNullOrWhiteSpace(avroFailedTask.message)
+                    ? "No message in Failed Task."
+                    : avroFailedTask.message;
+
+                // Gets the Exception.
+                _cause = GetCause(avroFailedTask.cause, ByteUtilities.ByteArraysToString(avroFailedTask.data));
+
+                // This is always empty, even in Java.
+                Description = Optional<string>.Empty();
+            }
+            else
+            {
+                Log.Log(Level.Warning, "no exception data");
+                Data = Optional<byte[]>.Empty();
+            }
+        }
+
+        public string Id { get; }
+        public string Message { get; set; }
+        public Optional<string> Description { get; set; }
+        public Optional<string> Reason { get; set; }
+        public Optional<byte[]> Data { get; set; }
+
+        public Optional<IActiveContext> GetActiveContext()
+        {
+            return _activeContext;
+        }
+
+        /// <summary>
+        /// Returns the Exception causing the Failed Task.
+        /// </summary>
+        /// <returns>the Exception causing the Failed Task.</returns>
+        /// <remarks>
+        /// If the Exception was caused by a control flow error (start, stop, suspend),
+        /// a <see cref="TaskClientCodeException"/> is expected.
+        /// If the original Exception was not serializable, a <see cref="NonSerializableTaskException"/> is expected.
+        /// If the Exception was missing, presumably caused by a failed Evaluator, a
+        /// <see cref="TaskExceptionMissingException"/> is expected.
+        /// </remarks>
+        public Exception AsError()
+        {
+            return _cause;
+        }
+
+        private static Exception GetCause(byte[] serializedCause, string originalTaskExceptionToString)
+        {
+            // TODO[JIRA REEF-1422]: Distinguish between Java Task Exception and missing Exception.
+            if (ByteUtilities.IsNullOrEmpty(serializedCause))
+            {
+                return new TaskExceptionMissingException(
+                    "Task failed without an Exception, presumably caused by an Exception failure. Please inspect the FailedTask message.");
+            }
+            try
+            {
+                return (Exception)ByteUtilities.DeserializeFromBinaryFormat(serializedCause);
+            }
+            catch (SerializationException se)
+            {
+                return NonSerializableTaskException.UnableToDeserialize(originalTaskExceptionToString, se);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeRunningTask.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeRunningTask.cs
new file mode 100644
index 0000000..e469202
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeRunningTask.cs
@@ -0,0 +1,64 @@
+// 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.
+
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Utilities;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class BridgeRunningTask : IRunningTask
+    {
+        private readonly IDriverServiceClient _driverServiceClient;
+
+        public string Id { get; }
+
+        public IActiveContext ActiveContext { get; }
+
+        public BridgeRunningTask(IDriverServiceClient driverServiceClient, string id, IActiveContext context)
+        {
+            _driverServiceClient = driverServiceClient;
+            Id = id;
+            ActiveContext = context;
+        }
+
+        public void Send(byte[] message)
+        {
+            _driverServiceClient.OnTaskMessage(Id, message);
+        }
+
+        public void Suspend(byte[] message)
+        {
+            _driverServiceClient.OnTaskSuspend(Id, Optional<byte[]>.Of(message));
+        }
+
+        public void Suspend()
+        {
+            _driverServiceClient.OnTaskSuspend(Id, Optional<byte[]>.Empty());
+        }
+
+        public void Dispose()
+        {
+            _driverServiceClient.OnTaskClose(Id, Optional<byte[]>.Empty());
+        }
+
+        public void Dispose(byte[] message)
+        {
+            _driverServiceClient.OnTaskClose(Id, Optional<byte[]>.Of(message));
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeSuspendedTask.cs
similarity index 60%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeSuspendedTask.cs
index 2f3352a..256a799 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeSuspendedTask.cs
@@ -15,12 +15,22 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Task;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class BridgeSuspendedTask : ISuspendedTask
     {
+        public byte[] Message { get; }
+        public string Id { get; }
+        public IActiveContext ActiveContext { get; }
+
+        public BridgeSuspendedTask(byte[] message, string id, IActiveContext activeContext)
+        {
+            Message = message;
+            Id = id;
+            ActiveContext = activeContext;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeTaskMessage.cs
similarity index 62%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeTaskMessage.cs
index 2f3352a..ed5d67e 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/BridgeTaskMessage.cs
@@ -15,12 +15,21 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Driver.Task;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    internal sealed class BridgeTaskMessage : ITaskMessage
     {
+        public byte[] Message { get; }
+        public string TaskId { get; }
+        public string MessageSourceId { get; }
+
+        public BridgeTaskMessage(byte[] message, string taskId, string messageSourceId)
+        {
+            Message = message;
+            TaskId = taskId;
+            MessageSourceId = messageSourceId;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/EvaluatorDescriptor.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/EvaluatorDescriptor.cs
new file mode 100644
index 0000000..f28dd89
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/EvaluatorDescriptor.cs
@@ -0,0 +1,57 @@
+// 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.
+
+using Org.Apache.REEF.Common.Catalog;
+using Org.Apache.REEF.Common.Evaluator;
+using Org.Apache.REEF.Common.Runtime;
+using Org.Apache.REEF.Driver.Evaluator;
+using System;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class EvaluatorDescriptor : IEvaluatorDescriptor
+    {
+        public INodeDescriptor NodeDescriptor { get; }
+
+        public EvaluatorType EvaluatorType => EvaluatorType.CLR;
+
+        public int Memory { get; }
+        public int VirtualCore { get; }
+
+        public string Rack
+        {
+            get
+            {
+                throw new NotImplementedException();
+            }
+        }
+
+        public RuntimeName RuntimeName { get; }
+
+        public EvaluatorDescriptor(
+            INodeDescriptor nodeDescriptor,
+            int memory,
+            int virtualCore,
+            RuntimeName runtimeName = RuntimeName.Local)
+        {
+            NodeDescriptor = nodeDescriptor;
+            Memory = memory;
+            VirtualCore = virtualCore;
+            RuntimeName = runtimeName;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/NodeDescriptor.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/NodeDescriptor.cs
new file mode 100644
index 0000000..101f924
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/Events/NodeDescriptor.cs
@@ -0,0 +1,58 @@
+// 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.
+
+using Org.Apache.REEF.Common.Catalog;
+using Org.Apache.REEF.Common.Catalog.Capabilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System.Net;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver.Events
+{
+    internal sealed class NodeDescriptor : INodeDescriptor
+    {
+        private static readonly Logger _log = Logger.GetLogger(typeof(NodeDescriptor));
+
+        public IPEndPoint InetSocketAddress { get; set; }
+        public string HostName { get; set; }
+        public CPU Cpu { get; set; }
+        public RAM Ram { get; set; }
+
+        public NodeDescriptor(IPEndPoint inetSocketAddress, string hostName, CPU cpu, RAM ram)
+        {
+            InetSocketAddress = inetSocketAddress;
+            HostName = hostName;
+            Cpu = cpu;
+            Ram = ram;
+        }
+
+        public NodeDescriptor(string ip, int port, string hostName, CPU cpu, RAM ram)
+        {
+            IPAddress address;
+            if (IPAddress.TryParse(ip, out address))
+            {
+                InetSocketAddress = new IPEndPoint(address, port);
+            }
+            else
+            {
+                _log.Log(Level.Error, "Could not parse ip address {0}", ip);
+            }
+            HostName = hostName;
+            Cpu = cpu;
+            Ram = ram;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/IDriverClientService.cs
similarity index 66%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/IDriverClientService.cs
index 2f3352a..6f30e4e 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/IDriverClientService.cs
@@ -14,13 +14,21 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <summary>
+    /// The interface that initializes the driver client.
+    /// </summary>
+    internal interface IDriverClientService
     {
+        /// <summary>
+        /// Start the driver client service.
+        /// </summary>
+        void Start();
+
+        /// <summary>
+        /// Await for termination.
+        /// </summary>
+        void AwaitTermination();
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/IDriverServiceClient.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/IDriverServiceClient.cs
new file mode 100644
index 0000000..1444a60
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Driver/IDriverServiceClient.cs
@@ -0,0 +1,128 @@
+// 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.
+
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities;
+using System;
+using System.Collections.Generic;
+using System.IO;
+
+namespace Org.Apache.REEF.Bridge.Core.Common.Driver
+{
+    /// <summary>
+    /// The interface for talking to the Java driver.
+    /// </summary>
+    internal interface IDriverServiceClient
+    {
+        /// <summary>
+        /// Initiate shutdown.
+        /// </summary>
+        void OnShutdown();
+
+        /// <summary>
+        /// Initiate shutdown with exception.
+        /// </summary>
+        /// <param name="ex">exception</param>
+        void OnShutdown(Exception ex);
+
+        /// <summary>
+        /// Set an alarm that will timeout it the given
+        /// amount of milliseconds.
+        /// </summary>
+        /// <param name="alarmId">Identifies the alarm</param>
+        /// <param name="timeoutMs">When to tigger the alarm in the future</param>
+        void OnSetAlarm(string alarmId, long timeoutMs);
+
+        /// <summary>
+        /// Request evalautor resources.
+        /// </summary>
+        /// <param name="evaluatorRequest">Request details</param>
+        void OnEvaluatorRequest(IEvaluatorRequest evaluatorRequest);
+
+        /// <summary>
+        /// Close the evaluator with the given identifier.
+        /// </summary>
+        /// <param name="evalautorId">Evaluator identifier</param>
+        void OnEvaluatorClose(string evalautorId);
+
+        /// <summary>
+        /// Submit the evalautor with the given configuration.
+        /// </summary>
+        /// <param name="evaluatorId">Evaluator identifier</param>
+        /// <param name="contextConfiguration">Context configuration</param>
+        /// <param name="serviceConfiguration">Service configuration</param>
+        /// <param name="taskConfiguration">Task configuration</param>
+        /// <param name="addFileList">Files that should be included</param>
+        /// <param name="addLibraryList">Libraries that should be included</param>
+        void OnEvaluatorSubmit(
+            string evaluatorId,
+            IConfiguration contextConfiguration,
+            Optional<IConfiguration> serviceConfiguration,
+            Optional<IConfiguration> taskConfiguration,
+            List<FileInfo> addFileList,
+            List<FileInfo> addLibraryList);
+
+        /// <summary>
+        /// Close the context with the given identifier.
+        /// </summary>
+        /// <param name="contextId">Context identifier</param>
+        void OnContextClose(string contextId);
+
+        /// <summary>
+        /// Submit a new context with the given configuration.
+        /// </summary>
+        /// <param name="contextId">Context identifier through which a new context will be created</param>
+        /// <param name="contextConfiguration">Configuration of the new (child) context</param>
+        void OnContextSubmitContext(string contextId, IConfiguration contextConfiguration);
+
+        /// <summary>
+        /// Submit a task via the given context.
+        /// </summary>
+        /// <param name="contextId">Context identifier</param>
+        /// <param name="taskConfiguration">Task configuration</param>
+        void OnContextSubmitTask(string contextId, IConfiguration taskConfiguration);
+
+        /// <summary>
+        /// Send a message to the given context.
+        /// </summary>
+        /// <param name="contextId">Context identifier</param>
+        /// <param name="message">Message to send</param>
+        void OnContextMessage(string contextId, byte[] message);
+
+        /// <summary>
+        /// Close the task with an optional message.
+        /// </summary>
+        /// <param name="taskId">Task identifier</param>
+        /// <param name="message">Optional message</param>
+        void OnTaskClose(string taskId, Optional<byte[]> message);
+
+        /// <summary>
+        /// Suspend the task with an optional message.
+        /// </summary>
+        /// <param name="taskId">Task identifier to suspend</param>
+        /// <param name="message">Optional message</param>
+        void OnTaskSuspend(string taskId, Optional<byte[]> message);
+
+        /// <summary>
+        /// Send a message to the task.
+        /// </summary>
+        /// <param name="taskId">Task identifier</param>
+        /// <param name="message">Message to send</param>
+        void OnTaskMessage(string taskId, byte[] message);
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Org.Apache.REEF.Bridge.Core.Common.csproj b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Org.Apache.REEF.Bridge.Core.Common.csproj
new file mode 100644
index 0000000..403e46e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Org.Apache.REEF.Bridge.Core.Common.csproj
@@ -0,0 +1,36 @@
+<!--
+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>
+  <PropertyGroup>
+    <AssemblyName>Org.Apache.REEF.Bridge.Core.Common</AssemblyName>
+    <Description>REEF Bridge Core Common</Description>
+    <PackageTags>REEF Bridge Core Common</PackageTags>
+    <TargetFramework>netstandard2.0</TargetFramework>
+  </PropertyGroup>
+  <Import Project="..\build.props" />
+  <ItemGroup>
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Proto\Org.Apache.REEF.Bridge.Core.Proto.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Tang\Org.Apache.REEF.Tang.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Utilities\Org.Apache.REEF.Utilities.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Common\Org.Apache.REEF.Common.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Driver\Org.Apache.REEF.Driver.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Wake\Org.Apache.REEF.Wake.csproj" />
+  </ItemGroup>
+  <Import Project="..\build.targets" />
+</Project>
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Properties/AssemblyInfo.cs
similarity index 86%
copy from lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Common/Properties/AssemblyInfo.cs
index e7b4768..6c75063 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Common/Properties/AssemblyInfo.cs
@@ -17,6 +17,5 @@
 
 using System.Runtime.CompilerServices;
 
-// Allow the bridge access to `internal` APIs
-[assembly: InternalsVisibleTo("Org.Apache.REEF.Bridge")]
-
+[assembly: InternalsVisibleTo("Org.Apache.REEF.Bridge.Core.Grpc")]
+[assembly: InternalsVisibleTo("Org.Apache.REEF.Bridge.Core.Driver")]
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Driver/DriverLauncher.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Driver/DriverLauncher.cs
new file mode 100644
index 0000000..8c45597
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Driver/DriverLauncher.cs
@@ -0,0 +1,90 @@
+// 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.
+using System;
+using System.IO;
+using Grpc.Core;
+using Grpc.Core.Logging;
+using Org.Apache.REEF.Bridge.Core.Grpc.Driver;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Implementations.Configuration;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Wake.Time;
+
+namespace Org.Apache.REEF.Bridge.Core.Driver
+{
+    /// <summary>
+    /// Used to launch the C# driver client.
+    /// </summary>
+    public sealed class DriverLauncher
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(DriverLauncher));
+
+        private readonly IConfigurationSerializer _configurationSerializer;
+
+        [Inject]
+        private DriverLauncher(IConfigurationSerializer configurationSerializer)
+        {
+            _configurationSerializer = configurationSerializer;
+        }
+
+        private IConfiguration GetDriverClientConfiguration(string configFile, string driverServicePort)
+        {
+            if (!File.Exists(configFile))
+            {
+                // If we reached this, we weren't able to find the configuration file.
+                var message = $"Unable to find brigde configuration. Paths checked: [\'{configFile}\']";
+
+                Log.Log(Level.Error, message);
+                throw new FileNotFoundException(message);
+            }
+            else
+            {
+                return Configurations.Merge(_configurationSerializer.FromFile(configFile),
+                    TangFactory.GetTang().NewConfigurationBuilder()
+                        .BindNamedParameter(typeof(DriverServicePort), driverServicePort)
+                        .Build());
+
+            }
+        }
+
+        public static void Main(string[] args)
+        {
+            if (args.Length != 2)
+            {
+                var message = $"Driver client launcher takes a two arguments. Found {args}";
+                Log.Log(Level.Error, message);
+                throw new ArgumentException(message);
+            }
+            else
+            {
+                Log.Log(Level.Info, "Launching driver client with driver service port {0} config file {1}", args[0], args[1]);
+            }
+            GrpcEnvironment.SetLogger(new ConsoleLogger());
+            Log.Log(Level.Info, "Path of executing assembly{0}", ClientUtilities.GetPathToExecutingAssembly());
+            var launcher = TangFactory.GetTang().NewInjector().GetInstance<DriverLauncher>();
+            var driverClientConfig = launcher.GetDriverClientConfiguration(args[0], args[1]);
+            var clock = TangFactory.GetTang().NewInjector(driverClientConfig).GetInstance<IClock>();
+            clock.Run();
+
+            Log.Log(Level.Info, "Driver client clock exit.");
+        }
+    }
+}
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Driver/Org.Apache.REEF.Bridge.Core.Driver.csproj b/lang/cs/Org.Apache.REEF.Bridge.Core.Driver/Org.Apache.REEF.Bridge.Core.Driver.csproj
new file mode 100644
index 0000000..107759a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Driver/Org.Apache.REEF.Bridge.Core.Driver.csproj
@@ -0,0 +1,34 @@
+<!--
+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>
+  <PropertyGroup>
+    <AssemblyName>Org.Apache.REEF.Bridge.Core.Driver</AssemblyName>
+    <Description>REEF Bridge Core Driver</Description>
+    <PackageTags>REEF Bridge Core Driver</PackageTags>
+  </PropertyGroup>
+
+  <Import Project="..\build.App.props" />
+  <ItemGroup>
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Grpc\Org.Apache.REEF.Bridge.Core.Grpc.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Tang\Org.Apache.REEF.Tang.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Common\Org.Apache.REEF.Common.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Wake\Org.Apache.REEF.Wake.csproj" />
+  </ItemGroup>
+  <Import Project="..\build.targets" />
+</Project>
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientLauncher.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientLauncher.cs
new file mode 100644
index 0000000..13b1f9d
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientLauncher.cs
@@ -0,0 +1,235 @@
+// 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.
+
+using Google.Protobuf;
+using Grpc.Core;
+using Org.Apache.REEF.Bridge.Core.Common.Client;
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime.Proto;
+using Org.Apache.REEF.Bridge.Core.Common.Driver;
+using Org.Apache.REEF.Bridge.Core.Grpc.Driver;
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Implementations.Configuration;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.IO;
+using System.Linq;
+using System.Threading;
+using System.Threading.Tasks;
+
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Client
+{
+    internal sealed class ClientLauncher : IClientLauncher
+    {
+        private const string SoFileNameExtension = ".so";
+        private const string DylibFileNameExtension = ".dylib";
+        private const string DllFileNameExtension = ".dll";
+        private const string ExeFileNameExtension = ".exe";
+        private const string JarFileNameExtension = ".jar";
+        private const string ConfigFileNameExtension = ".config";
+
+        private const string JavaClientLauncherClass = "org.apache.reef.bridge.client.grpc.ClientLauncher";
+
+        private const string DriverExe = "Org.Apache.REEF.Bridge.Core.Driver.exe";
+        private const string DriverDll = "Org.Apache.REEF.Bridge.Core.Driver.dll";
+
+        private static readonly string jsonSDK = @"
+{
+  ""runtimeOptions"": {
+        ""tfm"": ""netcoreapp2.0"",
+        ""framework"": {
+            ""name"": ""Microsoft.NETCore.App"",
+            ""version"": ""2.0.0""
+        }
+    }
+}
+";
+
+        private static readonly Logger Log = Logger.GetLogger(typeof(ClientLauncher));
+
+        private readonly ClientService _clientService;
+
+        private readonly DriverClientConfiguration _driverClientConfiguration;
+
+        private readonly REEFFileNames _reefFileNames;
+
+        private readonly IConfigurationSerializer _configurationSerializer;
+
+        private readonly JavaClientLauncher _javaClientLauncher;
+
+        private readonly Server _grpcServer;
+
+        private readonly int _grpcServerPort;
+
+        [Inject]
+        private ClientLauncher(
+            ClientService clientService,
+            JavaClientLauncher javaClientLauncher,
+            REEFFileNames reefFileNames,
+            IConfigurationSerializer configurationSerializer,
+            DriverClientParameters driverRuntimeProto,
+            IRuntimeProtoProvider runtimeProtoProvider)
+        {
+            _clientService = clientService;
+            _javaClientLauncher = javaClientLauncher;
+            _reefFileNames = reefFileNames;
+            _configurationSerializer = configurationSerializer;
+            _driverClientConfiguration = driverRuntimeProto.Proto;
+            runtimeProtoProvider.SetParameters(_driverClientConfiguration);
+            _grpcServer = new Server
+            {
+                Services = { BridgeClient.BindService(clientService) },
+                Ports = { new ServerPort("localhost", 0, ServerCredentials.Insecure) }
+            };
+            _grpcServer.Start();
+            Log.Log(Level.Info, "Server port any {0}", _grpcServer.Ports.Any());
+            foreach (var serverPort in _grpcServer.Ports)
+            {
+                Log.Log(Level.Info, "Server port {0}", serverPort.BoundPort);
+                _grpcServerPort = serverPort.BoundPort;
+            }
+        }
+
+        public void Dispose()
+        {
+            _grpcServer.ShutdownAsync();
+        }
+
+        public async Task<LauncherStatus> SubmitAsync(
+            IConfiguration driverAppConfiguration,
+            CancellationToken cancellationToken)
+        {
+            var driverClientConfiguration =
+                Configurations.Merge(driverAppConfiguration,
+                    DriverBridgeConfiguration.ConfigurationModule
+                        .Set(DriverBridgeConfiguration.DriverServiceClient, GenericType<DriverServiceClient>.Class)
+                        .Set(DriverBridgeConfiguration.DriverClientService, GenericType<DriverClientService>.Class)
+                        .Build());
+            var jobFolder = Directory.CreateDirectory(Path.Combine(Path.GetTempPath(), Guid.NewGuid().ToString()));
+            Log.Log(Level.Info, "Job folder {0}", jobFolder);
+            Directory.CreateDirectory(jobFolder.FullName);
+            AddDependencies(jobFolder, driverClientConfiguration);
+            // Launch command
+            if (File.Exists(DriverExe))
+            {
+                _driverClientConfiguration.DriverClientLaunchCommand = string.Format(
+                    @"cmd.exe /c {0} {1}",
+                    Path.Combine(_reefFileNames.GetGlobalFolderPath(), DriverExe),
+                    _reefFileNames.GetClrDriverConfigurationPath());
+            }
+            else
+            {
+                _driverClientConfiguration.DriverClientLaunchCommand = string.Format(
+                    @"dotnet {0} {1}",
+                    Path.Combine(_reefFileNames.GetGlobalFolderPath(), DriverDll),
+                    _reefFileNames.GetClrDriverConfigurationPath());
+            }
+
+            var driverClientConfigFile = Path.Combine(jobFolder.FullName, "driverclient.json");
+            using (var outputFile = new StreamWriter(driverClientConfigFile))
+            {
+                outputFile.Write(JsonFormatter.Default.Format(_driverClientConfiguration));
+            }
+            // Submit a new job
+            _clientService.Reset();
+            var task = _javaClientLauncher.LaunchAsync(JavaLoggingSetting.Info,
+                JavaClientLauncherClass,
+                new[] { driverClientConfigFile, _grpcServerPort.ToString() },
+                cancellationToken);
+            lock (_clientService)
+            {
+                while (!_clientService.IsDone && !cancellationToken.IsCancellationRequested)
+                {
+                    Monitor.Wait(_clientService, TimeSpan.FromMinutes(1));
+                }
+            }
+            await task;
+            return _clientService.LauncherStatus;
+        }
+
+        private void AddDependencies(FileSystemInfo jobFolder, IConfiguration driverClientConfiguration)
+        {
+            // driver client configuration
+            var driverClientConfigurationFile = Path.GetFullPath(Path.Combine(
+                jobFolder.FullName,
+                _reefFileNames.GetClrDriverConfigurationName()));
+            _configurationSerializer.ToFile(driverClientConfiguration, driverClientConfigurationFile);
+            _driverClientConfiguration.LocalFiles.Add(driverClientConfigurationFile);
+
+            /*
+            var jsonSDKFile = Path.Combine(jobFolder.FullName, "Org.Apache.REEF.Bridge.Core.Driver.runtimeconfig.json");
+            File.WriteAllText(jsonSDKFile, jsonSDK);
+            _driverClientConfiguration.GlobalFiles.Add(jsonSDKFile);
+            */
+
+            // resource files
+            var a = typeof(ClientLauncher).Assembly;
+            foreach (var name in a.GetManifestResourceNames())
+            {
+                Log.Log(Level.Info, "Extracting resource {0}", name);
+                var resource = a.GetManifestResourceStream(name);
+                using (var file = new FileStream(name, FileMode.Create, FileAccess.Write))
+                {
+                    resource.CopyTo(file);
+                }
+            }
+
+            var directory = ClientUtilities.GetPathToExecutingAssembly();
+            {
+                if (Directory.Exists(directory))
+                {
+                    // For input paths that are directories, extract only files of a predetermined type
+                    _driverClientConfiguration.GlobalFiles.Add(
+                        Directory.GetFiles(directory).Where(IsAssemblyToCopy));
+                }
+                else
+                {
+                    // Throw if a path input was not a file or a directory
+                    throw new FileNotFoundException($"Global Assembly Directory not Found: {directory}");
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns true, if the given file path references a DLL or EXE or JAR.
+        /// </summary>
+        /// <param name="filePath"></param>
+        /// <returns></returns>
+        private static bool IsAssemblyToCopy(string filePath)
+        {
+            var fileName = Path.GetFileName(filePath);
+            if (string.IsNullOrWhiteSpace(fileName))
+            {
+                return false;
+            }
+
+            var lowerCasePath = fileName.ToLower();
+            return lowerCasePath.EndsWith(DllFileNameExtension) ||
+                   lowerCasePath.EndsWith(ExeFileNameExtension) ||
+                   lowerCasePath.EndsWith(DylibFileNameExtension) ||
+                   lowerCasePath.EndsWith(SoFileNameExtension) ||
+                   lowerCasePath.EndsWith(ConfigFileNameExtension) ||
+                   lowerCasePath.EndsWith(JarFileNameExtension);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientLauncherFactory.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientLauncherFactory.cs
new file mode 100644
index 0000000..42fcedb
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientLauncherFactory.cs
@@ -0,0 +1,65 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client;
+using Org.Apache.REEF.Tang.Implementations.Configuration;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities;
+
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Client
+{
+    /// <summary>
+    /// Factory class for creating a ClientLauncher
+    /// </summary>
+    public static class ClientLauncherFactory
+    {
+        /// <summary>
+        /// Get a client launcher that does not include a client configuration, which
+        /// will default to all default client handlers e.g., <see cref="DefaultRunningJobHandler"/>
+        /// </summary>
+        /// <param name="runtimeConfiguration">One of YARN, Local, Azure Batch, etc. runtime configuration</param>
+        /// <param name="driverRuntimeConfiguration">The core driver runtime configuration <see cref="DriverRuntimeConfiguration"/></param>
+        /// <returns>ClientLauncher</returns>
+        public static IClientLauncher GetLauncher(
+            IConfiguration runtimeConfiguration,
+            IConfiguration driverRuntimeConfiguration)
+        {
+            return GetLauncher(runtimeConfiguration, driverRuntimeConfiguration, Optional<IConfiguration>.Empty());
+        }
+
+        /// <summary>
+        /// Get a client launcher that optionally includes a client configuration.
+        /// </summary>
+        /// <param name="runtimeConfiguration">One of YARN, Local, Azure Batch, etc. runtime configuration</param>
+        /// <param name="driverRuntimeConfiguration">The core driver runtime configuration <see cref="DriverRuntimeConfiguration"/></param>
+        /// <param name="clientConfiguration">Client configuration <see cref="ClientConfiguration"/></param>
+        /// <returns></returns>
+        public static IClientLauncher GetLauncher(
+            IConfiguration runtimeConfiguration,
+            IConfiguration driverRuntimeConfiguration,
+            Optional<IConfiguration> clientConfiguration)
+        {
+            return TangFactory.GetTang()
+                .NewInjector(Configurations.Merge(
+                    runtimeConfiguration,
+                    driverRuntimeConfiguration,
+                    clientConfiguration.OrElse(TangFactory.GetTang().NewConfigurationBuilder().Build())))
+                .GetInstance<ClientLauncher>();
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientService.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientService.cs
new file mode 100644
index 0000000..cc56c6e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/ClientService.cs
@@ -0,0 +1,216 @@
+// 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.
+
+using Google.Protobuf;
+using Grpc.Core;
+using Org.Apache.REEF.Bridge.Core.Common.Client;
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config;
+using Org.Apache.REEF.Bridge.Core.Common.Client.Events;
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Threading;
+using System.Threading.Tasks;
+using Void = Org.Apache.REEF.Bridge.Core.Proto.Void;
+
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Client
+{
+    /// <summary>
+    /// gRPC based client service.
+    /// </summary>
+    internal sealed class ClientService : BridgeClient.BridgeClientBase, IClientService
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(ClientService));
+
+        private static readonly Void Void = new Void();
+
+        private readonly IObserver<ISubmittedJob> _submittedJobHandler;
+
+        private readonly IObserver<IRunningJob> _runningJobHandler;
+
+        private readonly IObserver<ICompletedJob> _completedJobHandler;
+
+        private readonly IObserver<IFailedJob> _failedJobHandler;
+
+        private readonly IObserver<IJobMessage> _jobMessageHandler;
+
+        private readonly IObserver<IFailedRuntime> _failedRuntimeHandler;
+
+        private readonly IObserver<IWakeError> _wakeErrorHandler;
+
+        private REEFClient.REEFClientClient _clientStub = null;
+
+        private string _jobId = "unknown";
+
+        [Inject]
+        private ClientService(
+            [Parameter(Value = typeof(ClientParameters.SubmittedJobHandler))] IObserver<ISubmittedJob> submittedJobHandler,
+            [Parameter(Value = typeof(ClientParameters.RunningJobHandler))] IObserver<IRunningJob> runningJobHandler,
+            [Parameter(Value = typeof(ClientParameters.CompletedJobHandler))] IObserver<ICompletedJob> completedJobHandler,
+            [Parameter(Value = typeof(ClientParameters.FailedJobHandler))] IObserver<IFailedJob> failedJobHandler,
+            [Parameter(Value = typeof(ClientParameters.JobMessageHandler))] IObserver<IJobMessage> jobMessageHandler,
+            [Parameter(Value = typeof(ClientParameters.FailedRuntimeHandler))] IObserver<IFailedRuntime> failedRuntimeHandler,
+            [Parameter(Value = typeof(ClientParameters.WakeErrorHandler))] IObserver<IWakeError> wakeErrorHandler)
+        {
+            _submittedJobHandler = submittedJobHandler;
+            _runningJobHandler = runningJobHandler;
+            _completedJobHandler = completedJobHandler;
+            _failedJobHandler = failedJobHandler;
+            _jobMessageHandler = jobMessageHandler;
+            _failedRuntimeHandler = failedRuntimeHandler;
+            _wakeErrorHandler = wakeErrorHandler;
+            LauncherStatus = LauncherStatus.InitStatus;
+        }
+
+        public bool IsDone => LauncherStatus.IsDone;
+
+        public void Reset()
+        {
+            LauncherStatus = LauncherStatus.InitStatus;
+        }
+
+        public LauncherStatus LauncherStatus { get; private set; }
+
+        public void Close(byte[] message = null)
+        {
+            try
+            {
+                _clientStub?.DriverControlHandler(new DriverControlOp()
+                {
+                    JobId = _jobId,
+                    Message = message == null ? ByteString.Empty : ByteString.CopyFrom(message),
+                    Operation = DriverControlOp.Types.Operation.Close
+                });
+            }
+            catch (Exception e)
+            {
+                Log.Log(Level.Warning, "exception occurred when trying to close job", e);
+            }
+            LauncherStatus = LauncherStatus.ForceCloseStatus;
+            _clientStub = null;
+        }
+
+        public void Send(byte[] message)
+        {
+            if (_clientStub != null)
+            {
+                _clientStub.DriverControlHandler(new DriverControlOp()
+                {
+                    JobId = _jobId,
+                    Message = ByteString.CopyFrom(message),
+                    Operation = DriverControlOp.Types.Operation.Message
+                });
+            }
+            else
+            {
+                throw new IllegalStateException("Client service is closed");
+            }
+        }
+
+        public override Task<Void> RegisterREEFClient(REEFClientRegistration request, ServerCallContext context)
+        {
+            Log.Log(Level.Info, "REEF Client registered on port {0}", request.Port);
+            Channel driverServiceChannel = new Channel("127.0.0.1", (int)request.Port, ChannelCredentials.Insecure);
+            _clientStub = new REEFClient.REEFClientClient(driverServiceChannel);
+            return Task.FromResult(Void);
+        }
+
+        public override Task<Void> JobMessageHandler(JobMessageEvent request, ServerCallContext context)
+        {
+            Log.Log(Level.Info, "Job message from job id {0}", request.JobId);
+            _jobMessageHandler.OnNext(new JobMessage(request.JobId, request.Message.ToByteArray()));
+            return Task.FromResult(Void);
+        }
+
+        public override Task<Void> JobSumittedHandler(JobSubmittedEvent request, ServerCallContext context)
+        {
+            Log.Log(Level.Info, "Job id {0} submitted", request.JobId);
+            UpdateStatusAndNotify(LauncherStatus.SubmittedStatus);
+            _submittedJobHandler.OnNext(new SubmittedJob(request.JobId));
+            _jobId = request.JobId;
+            return Task.FromResult(Void);
+        }
+
+        public override Task<Void> JobRunningHandler(JobRunningEvent request, ServerCallContext context)
+        {
+            Log.Log(Level.Info, "Job id {0} running", request.JobId);
+            UpdateStatusAndNotify(LauncherStatus.RunningStatus);
+            _runningJobHandler.OnNext(new RunningJob(this, request.JobId));
+            return Task.FromResult(Void);
+        }
+
+        public override Task<Void> JobCompletedHandler(JobCompletedEvent request, ServerCallContext context)
+        {
+            if (IsDone) return Task.FromResult(Void);
+            Log.Log(Level.Info, "Job id {0} completed", request.JobId);
+            UpdateStatusAndNotify(LauncherStatus.CompletedStatus);
+            _completedJobHandler.OnNext(new CompletedJob(request.JobId));
+            return Task.FromResult(Void);
+        }
+
+        public override Task<Void> JobFailedHandler(JobFailedEvent request, ServerCallContext context)
+        {
+            if (IsDone) return Task.FromResult(Void);
+            Log.Log(Level.Info, "Job id {0} failed on {1}", request.JobId, request.Exception.Name);
+            var jobFailedEvent = new FailedJob(request.JobId,
+                request.Exception.Message,
+                request.Exception.Data.ToByteArray());
+            UpdateStatusAndNotify(LauncherStatus.Failed(jobFailedEvent.AsError()));
+            _failedJobHandler.OnNext(jobFailedEvent);
+            return Task.FromResult(Void);
+        }
+
+        public override Task<Void> RuntimeErrorHandler(ExceptionInfo request, ServerCallContext context)
+        {
+            if (!IsDone)
+            {
+                Log.Log(Level.Info, "Runtime error {0}", request.Message);
+                UpdateStatusAndNotify(LauncherStatus.FailedStatus);
+                _failedRuntimeHandler.OnNext(new FailedRuntime(_jobId, request.Message, request.Data.ToByteArray()));
+            }
+
+            return Task.FromResult(Void);
+        }
+
+        public override Task<Void> WakeErrorHandler(ExceptionInfo request, ServerCallContext context)
+        {
+            if (!IsDone)
+            {
+                Log.Log(Level.Info, "Wake error {0}", request.Message);
+                UpdateStatusAndNotify(LauncherStatus.FailedStatus);
+                _wakeErrorHandler.OnNext(new WakeError(_jobId,
+                    request.Message,
+                    Optional<byte[]>.Of(request.Data.ToByteArray())));
+            }
+
+            return Task.FromResult(Void);
+        }
+
+        private void UpdateStatusAndNotify(LauncherStatus status)
+        {
+            lock (this)
+            {
+                LauncherStatus = status;
+                Monitor.PulseAll(this);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/DriverClientParameters.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/DriverClientParameters.cs
new file mode 100644
index 0000000..7f435fb
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Client/DriverClientParameters.cs
@@ -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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config;
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using System.Collections.Generic;
+
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Client
+{
+    internal sealed class DriverClientParameters
+    {
+        public DriverClientConfiguration Proto { get; }
+
+        [Inject]
+        private DriverClientParameters(
+            OsType osType,
+            [Parameter(Value = typeof(DriverRuntimeParameters.JobId))] string jobId,
+            [Parameter(Value = typeof(DriverRuntimeParameters.JobSubmissionDirectory))] string jobSubmissionDirectory,
+            [Parameter(Value = typeof(DriverRuntimeParameters.DriverCpuCores))] int driverCpuCores,
+            [Parameter(Value = typeof(DriverRuntimeParameters.DriverMemory))] int driverMemory,
+            [Parameter(Value = typeof(DriverRuntimeParameters.GlobalAssemblies))] ISet<string> globalAssemblies,
+            [Parameter(Value = typeof(DriverRuntimeParameters.LocalAssemblies))] ISet<string> localAssemblies,
+            [Parameter(Value = typeof(DriverRuntimeParameters.GlobalFiles))] ISet<string> globalFiles,
+            [Parameter(Value = typeof(DriverRuntimeParameters.LocalFiles))] ISet<string> localFiles,
+            [Parameter(Value = typeof(DriverRuntimeParameters.EnableDriverRestart))] bool enableDriverRestart,
+            [Parameter(Value = typeof(DriverRuntimeParameters.TcpPortRangeBegin))] int tcpPortRangeBegin,
+            [Parameter(Value = typeof(DriverRuntimeParameters.TcpPortRangeCount))] int tcpPortRangeCount,
+            [Parameter(Value = typeof(DriverRuntimeParameters.TcpPortRangeTryCount))] int tcpPortRangeTryCount,
+            [Parameter(Value = typeof(DriverRuntimeParameters.RestartEvaluatorRecoverySeconds))] int restartEvaluatorRecoverySeconds)
+        {
+            Proto = new DriverClientConfiguration()
+            {
+                Jobid = jobId,
+                DriverJobSubmissionDirectory = jobSubmissionDirectory,
+                CpuCores = (uint)driverCpuCores,
+                MemoryMb = (uint)driverMemory,
+                DriverRestartEnable = enableDriverRestart,
+                DriverRestartEvaluatorRecoverySeconds = (uint)restartEvaluatorRecoverySeconds,
+                TcpPortRangeBegin = (uint)tcpPortRangeBegin,
+                TcpPortRangeCount = (uint)tcpPortRangeCount,
+                TcpPortRangeTryCount = (uint)tcpPortRangeTryCount
+            };
+            Proto.GlobalLibraries.Add(globalAssemblies);
+            Proto.GlobalFiles.Add(globalFiles);
+            Proto.LocalLibraries.Add(localAssemblies);
+            Proto.LocalFiles.Add(localFiles);
+            switch (osType.Type)
+            {
+                case OsType.Os.Windows:
+                    Proto.OperatingSystem = DriverClientConfiguration.Types.OS.Windows;
+                    break;
+
+                case OsType.Os.Linux:
+                    Proto.OperatingSystem = DriverClientConfiguration.Types.OS.Linux;
+                    break;
+
+                default:
+                    throw new IllegalStateException("Unknown operating system type " + osType.Type);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverClientService.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverClientService.cs
new file mode 100644
index 0000000..79a6a75
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverClientService.cs
@@ -0,0 +1,738 @@
+// 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.
+
+using Grpc.Core;
+using Org.Apache.REEF.Bridge.Core.Common.Driver;
+using Org.Apache.REEF.Bridge.Core.Common.Driver.Events;
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Common.Catalog.Capabilities;
+using Org.Apache.REEF.Common.Exceptions;
+using Org.Apache.REEF.Common.Runtime;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Runtime.Serialization;
+using System.Threading.Tasks;
+using Void = Org.Apache.REEF.Bridge.Core.Proto.Void;
+
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Driver
+{
+    internal class DriverClientService : DriverClient.DriverClientBase, IDriverClientService
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(DriverClientService));
+
+        private static readonly Void Void = new Void();
+
+        private readonly object _lock = new object();
+
+        private readonly Server _server;
+
+        private readonly int _serverPort;
+
+        private readonly IInjector _injector;
+
+        private readonly BridgeClock _bridgeClock;
+
+        private readonly DriverServiceClient _driverServiceClient;
+
+        private readonly IDictionary<string, BridgeActiveContext> _activeContexts =
+            new Dictionary<string, BridgeActiveContext>();
+
+        private readonly IDictionary<string, BridgeRunningTask> _runningTasks =
+            new Dictionary<string, BridgeRunningTask>();
+
+        private DriverBridge _driverBridge;
+
+        private bool _receivedStartEvent;
+
+        private bool IsIdle
+        {
+            get
+            {
+                lock (_lock)
+                {
+                    return _receivedStartEvent && DriverBridge.IsIdle && _bridgeClock.IsIdle();
+                }
+            }
+        }
+
+        [Inject]
+        private DriverClientService(
+            IInjector injector,
+            BridgeClock bridgeClock,
+            DriverServiceClient driverServiceClient)
+        {
+            _receivedStartEvent = false;
+            _injector = injector;
+            _bridgeClock = bridgeClock;
+            _driverServiceClient = driverServiceClient;
+            _server = new Server
+            {
+                Services = { DriverClient.BindService(this) },
+                Ports = { new ServerPort("localhost", 0, ServerCredentials.Insecure) }
+            };
+            _server.Start();
+            foreach (var serverPort in _server.Ports)
+            {
+                Logger.Log(Level.Info, "Server port {0}", serverPort.BoundPort);
+                _serverPort = serverPort.BoundPort;
+            }
+
+            Logger.Log(Level.Info, "Client service started on port {0}", _serverPort);
+        }
+
+        #region Driver Client Service implementation
+
+        public void Start()
+        {
+            try
+            {
+                // could throw application level exception
+                _driverBridge = _injector.GetInstance<DriverBridge>();
+                _driverServiceClient.RegisterDriverClientService("localhost", _serverPort);
+            }
+            catch (Exception ex)
+            {
+                Logger.Log(Level.Error, "Driver application creation error", ex);
+                _driverServiceClient.RegisterDriverClientService(ex);
+                _server.ShutdownAsync();
+            }
+        }
+
+        public void AwaitTermination()
+        {
+            Logger.Log(Level.Info, "Awaiting client server termination");
+            _server.ShutdownTask.Wait();
+            Logger.Log(Level.Info, "Client server terminated");
+        }
+
+        #endregion Driver Client Service implementation
+
+        public override async Task<Void> AlarmTrigger(AlarmTriggerInfo request, ServerCallContext context)
+        {
+            await _bridgeClock.OnNextAsync(new BridgeClock.AlarmInfo
+            {
+                AlarmId = request.AlarmId,
+                Timestamp = request.Timestamp
+            });
+            return Void;
+        }
+
+        public override Task<IdleStatus> IdlenessCheckHandler(Void request, ServerCallContext context)
+        {
+            return Task.FromResult(new IdleStatus
+            {
+                IsIdle = IsIdle,
+                Reason = "Driver client"
+            });
+        }
+
+        public override async Task<Void> StartHandler(StartTimeInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Received start event at time {0}", request.StartTime);
+                await _driverBridge.DispatchStartEventAsync(new BridgeDriverStarted(new DateTime(request.StartTime)));
+                _receivedStartEvent = true;
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        [System.Diagnostics.CodeAnalysis.SuppressMessage("Await.Warning", "CS4014:Await.Warning")]
+        public override async Task<ExceptionInfo> StopHandler(StopTimeInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Received stop event at time {0}", request.StopTime);
+                await _driverBridge.DispatchStopEvent(new BridgeDriverStopped(new DateTime(request.StopTime)));
+            }
+            catch (Exception ex)
+            {
+                Logger.Log(Level.Error, "Driver stop handler error", ex);
+                return GrpcUtils.SerializeException(ex);
+            }
+            finally
+            {
+                /* Do not await on shutdown async, which will cause a deadlock since
+                 * shutdown waits for this method to return.*/
+                _server.ShutdownAsync();
+            }
+            Logger.Log(Level.Info, "Clean stop handler execution");
+            return new ExceptionInfo()
+            {
+                NoError = true
+            };
+        }
+
+        public override async Task<Void> ActiveContextHandler(ContextInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Active context event id {0}", request.ContextId);
+                var activeContext = GetOrCreateActiveContext(request);
+                await _driverBridge.DispatchActiveContextEvent(activeContext);
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> ClosedContextHandler(ContextInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Closed context event id {0}", request.ContextId);
+                await _driverBridge.DispatchClosedContextEvent(CreateClosedContextAndForget(request));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> ContextMessageHandler(ContextMessageInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Context message event id {0}", request.ContextId);
+                await _driverBridge.DispatchContextMessageEvent(
+                    new BridgeContextMessage(
+                        request.ContextId,
+                        request.MessageSourceId,
+                        request.Payload.ToByteArray()));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> FailedContextHandler(ContextInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Failed context event id {0}", request.ContextId);
+                BridgeActiveContext activeContext;
+                BridgeActiveContext parentContext = null;
+                lock (_lock)
+                {
+                    if (_activeContexts.TryGetValue(request.ContextId, out activeContext))
+                    {
+                        _activeContexts.Remove(request.ContextId);
+                        parentContext = activeContext.ParentId.IsPresent()
+                            ? _activeContexts[activeContext.ParentId.Value]
+                            : null;
+                    }
+                }
+
+                if (activeContext != null)
+                {
+                    await _driverBridge.DispatchFailedContextEvent(
+                        new BridgeFailedContext(
+                            activeContext.Id,
+                            activeContext.EvaluatorId,
+                            CreateEvaluatorDescriptor(request.EvaluatorDescriptorInfo),
+                            Optional<IActiveContext>.OfNullable(parentContext)));
+                }
+                else
+                {
+                    Logger.Log(Level.Error, "unknown failed context {0}", request.ContextId);
+                }
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        // Evaluator handlers
+
+        public override async Task<Void> AllocatedEvaluatorHandler(EvaluatorInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Allocated evaluator id {0}", request.EvaluatorId);
+                var evaluatorDescriptor = CreateEvaluatorDescriptor(request.DescriptorInfo);
+                await _driverBridge.DispatchAllocatedEvaluatorEventAsync(
+                    new BridgeAllocatedEvaluator(request.EvaluatorId, _driverServiceClient, evaluatorDescriptor));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> CompletedEvaluatorHandler(EvaluatorInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Completed evaluator id {0}", request.EvaluatorId);
+                await _driverBridge.DispatchCompletedEvaluatorEvent(new BridgeCompletedEvaluator(request.EvaluatorId));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> FailedEvaluatorHandler(EvaluatorInfo info, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Failed evaluator id {0}", info.EvaluatorId);
+                await _driverBridge.DispatchFailedEvaluatorEvent(CreateFailedEvaluator(info));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        #region Task handlers
+
+        public override async Task<Void> RunningTaskHandler(TaskInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Running task {0}", request.TaskId);
+                var task = GetOrCreateRunningTask(request);
+                Logger.Log(Level.Info, "Dispatch running task {0}", task.Id);
+                await _driverBridge.DispatchRunningTaskEvent(task);
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> CompletedTaskHandler(TaskInfo request, ServerCallContext context)
+        {
+            try
+            {
+                lock (_lock)
+                {
+                    Logger.Log(Level.Info, "Completed task {0}", request.TaskId);
+                    _runningTasks.Remove(request.TaskId);
+                }
+                var activeContext = GetOrCreateActiveContext(request.Context);
+                await _driverBridge.DispatchCompletedTaskEvent(new BridgeCompletedTask(request.Result.ToByteArray(),
+                    request.TaskId,
+                    activeContext));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> FailedTaskHandler(TaskInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Failed task {0}", request.TaskId);
+                var failedTask = CreateFailedTaskAndForget(request.TaskId, request.Exception,
+                    Optional<IActiveContext>.OfNullable(GetOrCreateActiveContext(request.Context)));
+                Logger.Log(Level.Info, "Dispatch failed task {0}", request.TaskId);
+                await _driverBridge.DispatchFailedTaskEvent(failedTask);
+            }
+            catch (Exception ex)
+            {
+                Logger.Log(Level.Warning, "Failed task exception", ex);
+                _bridgeClock.Dispose(ex);
+            }
+            Logger.Log(Level.Info, "Done with failed task handler");
+            return Void;
+        }
+
+        public override async Task<Void> TaskMessageHandler(TaskMessageInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Message from task {0}", request.TaskId);
+                await _driverBridge.DispatchTaskMessageEvent(new BridgeTaskMessage(request.Payload.ToByteArray(),
+                    request.TaskId,
+                    request.MessageSourceId));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> SuspendedTaskHandler(TaskInfo request, ServerCallContext context)
+        {
+            try
+            {
+                Logger.Log(Level.Info, "Suspended task {0}", request.TaskId);
+                var activeContext = GetOrCreateActiveContext(request.Context);
+                await _driverBridge.DispatchSuspendedTaskEvent(new BridgeSuspendedTask(request.Result.ToByteArray(),
+                    request.TaskId,
+                    activeContext));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        #endregion Task handlers
+
+        #region Client handlers
+
+        public override async Task<Void> ClientCloseHandler(Void request, ServerCallContext context)
+        {
+            try
+            {
+                await _driverBridge.DispatchClientCloseEvent();
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> ClientCloseWithMessageHandler(ClientMessageInfo request, ServerCallContext context)
+        {
+            try
+            {
+                await _driverBridge.DispatchClientCloseWithMessageEvent(request.Payload.ToByteArray());
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> ClientMessageHandler(ClientMessageInfo request, ServerCallContext context)
+        {
+            try
+            {
+                await _driverBridge.DispatchClientMessageEvent(request.Payload.ToByteArray());
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        #endregion Client handlers
+
+        #region Driver Restart Handlers
+
+        public override async Task<Void> DriverRestartHandler(DriverRestartInfo request, ServerCallContext context)
+        {
+            try
+            {
+                ISet<string> expectedEvaluatorIds = new HashSet<string>(request.ExpectedEvaluatorIds);
+                await _driverBridge.DispatchDriverRestartedEvent(new BridgeDriverRestarted(
+                    new DateTime(request.StartTime.StartTime),
+                    expectedEvaluatorIds,
+                    (int)request.ResubmissionAttempts));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> DriverRestartActiveContextHandler(ContextInfo request, ServerCallContext context)
+        {
+            try
+            {
+                var activeContext = GetOrCreateActiveContext(request);
+                await _driverBridge.DispatchDriverRestartActiveContextEvent(activeContext);
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> DriverRestartRunningTaskHandler(TaskInfo request, ServerCallContext context)
+        {
+            try
+            {
+                var runningTask = GetOrCreateRunningTask(request);
+                await _driverBridge.DispatchDriverRestartRunningTaskEvent(runningTask);
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+
+            return Void;
+        }
+
+        public override async Task<Void> DriverRestartCompletedHandler(DriverRestartCompletedInfo request, ServerCallContext context)
+        {
+            try
+            {
+                await _driverBridge.DispatchDriverRestartCompletedEvent(
+                    new BridgeDriverRestartCompleted(new DateTime(request.CompletionTime.StopTime),
+                        request.IsTimedOut));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        public override async Task<Void> DriverRestartFailedEvaluatorHandler(EvaluatorInfo request, ServerCallContext context)
+        {
+            try
+            {
+                await _driverBridge.DispatchDriverRestartFailedEvaluatorEvent(CreateFailedEvaluator(request));
+            }
+            catch (Exception ex)
+            {
+                _bridgeClock.Dispose(ex);
+            }
+            return Void;
+        }
+
+        #endregion Driver Restart Handlers
+
+        #region helper methods
+
+        private BridgeActiveContext GetOrCreateActiveContext(ContextInfo info)
+        {
+            lock (_lock)
+            {
+                if (_activeContexts.TryGetValue(info.ContextId, out BridgeActiveContext context))
+                {
+                    Logger.Log(Level.Verbose, "Context already exists, use it: {0}", info.ContextId);
+                    return context;
+                }
+
+                Logger.Log(Level.Verbose, "Create active context {0}", info.ContextId);
+
+                context = new BridgeActiveContext(
+                    _driverServiceClient,
+                    info.ContextId,
+                    info.EvaluatorId,
+                    Optional<string>.OfNullable(info.ParentId.Length == 0 ? null : info.ParentId),
+                    CreateEvaluatorDescriptor(info.EvaluatorDescriptorInfo));
+
+                _activeContexts.Add(info.ContextId, context);
+
+                return context;
+            }
+        }
+
+        private BridgeRunningTask GetOrCreateRunningTask(TaskInfo task)
+        {
+            lock (_lock)
+            {
+                if (_runningTasks.TryGetValue(task.TaskId, out BridgeRunningTask runningTask))
+                {
+                    return runningTask;
+                }
+                var activeContext = GetOrCreateActiveContext(task.Context);
+                runningTask = new BridgeRunningTask(_driverServiceClient, task.TaskId, activeContext);
+                _runningTasks[task.TaskId] = runningTask;
+                return runningTask;
+            }
+        }
+
+        private BridgeFailedTask CreateFailedTaskAndForget(string taskId, ExceptionInfo info, Optional<IActiveContext> context)
+        {
+            lock (_lock)
+            {
+                if (_runningTasks.TryGetValue(taskId, out BridgeRunningTask task))
+                {
+                    Logger.Log(Level.Info, "Create failed task {0}", taskId);
+                    _runningTasks.Remove(taskId);
+                    return new BridgeFailedTask(
+                        Optional<IActiveContext>.Of(task.ActiveContext),
+                        task.Id,
+                        info.Message,
+                        info.Data.ToByteArray());
+                }
+                else
+                {
+                    Logger.Log(Level.Warning, "Create unknown failed task {0} with data {1}",
+                        taskId, !info.Data.IsEmpty);
+                    return new BridgeFailedTask(
+                        context,
+                        taskId,
+                        info.Message,
+                        info.Data.ToByteArray());
+                }
+            }
+        }
+
+        private BridgeFailedContext CreateFailedContextAndForget(EvaluatorInfo info, string contextId)
+        {
+            lock (_lock)
+            {
+                var evaluatorId = info.EvaluatorId;
+                if (_activeContexts.TryGetValue(contextId, out BridgeActiveContext activeContext))
+                {
+                    _activeContexts.Remove(contextId);
+                    var parentContext = activeContext.ParentId.IsPresent()
+                        ? _activeContexts[activeContext.ParentId.Value]
+                        : null;
+                    return new BridgeFailedContext(contextId,
+                        evaluatorId,
+                        activeContext.EvaluatorDescriptor,
+                        Optional<IActiveContext>.OfNullable(parentContext));
+                }
+                else
+                {
+                    return new BridgeFailedContext(contextId,
+                        evaluatorId,
+                        CreateEvaluatorDescriptor(info.DescriptorInfo),
+                        Optional<IActiveContext>.Empty());
+                }
+            }
+        }
+
+        private BridgeClosedContext CreateClosedContextAndForget(ContextInfo request)
+        {
+            lock (_lock)
+            {
+                if (_activeContexts.TryGetValue(request.ContextId, out BridgeActiveContext activeContext))
+                {
+                    _activeContexts.Remove(request.ContextId);
+                    var parentContext = _activeContexts[activeContext.ParentId.Value];
+                    return new BridgeClosedContext(
+                        activeContext.Id,
+                        activeContext.EvaluatorId,
+                        Optional<string>.Of(parentContext.Id),
+                        CreateEvaluatorDescriptor(request.EvaluatorDescriptorInfo),
+                        parentContext);
+                }
+                else
+                {
+                    Logger.Log(Level.Error, "Unknown context {0}", request.ContextId);
+                    return new BridgeClosedContext(
+                        request.ContextId,
+                        request.EvaluatorId,
+                        Optional<string>.Empty(),
+                        CreateEvaluatorDescriptor(request.EvaluatorDescriptorInfo),
+                        null);
+                }
+            }
+        }
+
+        private BridgeFailedEvaluator CreateFailedEvaluator(EvaluatorInfo info)
+        {
+            try
+            {
+                var failedContexts = info.Failure.FailedContexts.Select(contextId =>
+                    CreateFailedContextAndForget(info, contextId)).Cast<IFailedContext>().ToList();
+                var failedTask = string.IsNullOrEmpty(info.Failure.FailedTaskId)
+                    ? Optional<IFailedTask>.Empty()
+                    : Optional<IFailedTask>.Of(CreateFailedTaskAndForget(info.Failure.FailedTaskId,
+                        info.Failure.Exception,
+                        Optional<IActiveContext>.Empty()));
+
+                return new BridgeFailedEvaluator(
+                    info.EvaluatorId,
+                    CreateEvaluatorException(info),
+                    failedContexts,
+                    failedTask);
+            }
+            catch (Exception ex)
+            {
+                Logger.Log(Level.Error, "could not create failed evaluator", ex);
+                throw;
+            }
+        }
+
+        private static EvaluatorException CreateEvaluatorException(EvaluatorInfo eval)
+        {
+            var failureInfo = eval.Failure;
+            var errorBytes = failureInfo.Exception.Data.ToByteArray();
+            if (errorBytes == null || errorBytes.Length == 0)
+            {
+                Logger.Log(Level.Error, "Exception without object details: {0}", failureInfo.Exception.Message);
+                return new EvaluatorException(
+                    eval.EvaluatorId,
+                    failureInfo.Exception.Message,
+                    string.Join(";", failureInfo.Exception.StackTrace));
+            }
+            // When the Exception originates from the C# side.
+            Exception inner;
+            try
+            {
+                inner = (Exception)ByteUtilities.DeserializeFromBinaryFormat(errorBytes);
+            }
+            catch (SerializationException se)
+            {
+                inner = NonSerializableEvaluatorException.UnableToDeserialize(
+                    "Exception from Evaluator was not able to be deserialized, returning a NonSerializableEvaluatorException.",
+                    se);
+            }
+            return new EvaluatorException(eval.EvaluatorId, inner.Message, inner);
+        }
+
+        private static IEvaluatorDescriptor CreateEvaluatorDescriptor(EvaluatorDescriptorInfo descriptorInfo)
+        {
+            var runtimeName = RuntimeName.Local;
+            if (!string.IsNullOrWhiteSpace(descriptorInfo.RuntimeName) &&
+                !Enum.TryParse(descriptorInfo.RuntimeName, true, out runtimeName))
+            {
+                throw new ArgumentException($"Unknown runtime name received {descriptorInfo.RuntimeName}");
+            }
+
+            if (descriptorInfo.NodeDescriptorInfo == null)
+            {
+                Logger.Log(Level.Warning, "Node Descriptor not present in Evalautor Descriptor");
+                return null;
+            }
+            var nodeDescriptor = new NodeDescriptor(
+                descriptorInfo.NodeDescriptorInfo.IpAddress,
+                descriptorInfo.NodeDescriptorInfo.Port,
+                descriptorInfo.NodeDescriptorInfo.HostName,
+                new CPU(descriptorInfo.Cores),
+                new RAM(descriptorInfo.Memory));
+            return new EvaluatorDescriptor(nodeDescriptor, descriptorInfo.Memory, descriptorInfo.Cores, runtimeName);
+        }
+
+        #endregion helper methods
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverServiceClient.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverServiceClient.cs
new file mode 100644
index 0000000..bc43e33
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverServiceClient.cs
@@ -0,0 +1,261 @@
+// 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.
+
+using Google.Protobuf;
+using Grpc.Core;
+using Org.Apache.REEF.Bridge.Core.Common.Driver;
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Common.Exceptions;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Linq;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Driver
+{
+    internal class DriverServiceClient : IDriverServiceClient
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(DriverServiceClient));
+
+        private readonly IConfigurationSerializer _configurationSerializer;
+
+        private readonly DriverService.DriverServiceClient _driverServiceStub;
+
+        [Inject]
+        private DriverServiceClient(
+            IConfigurationSerializer configurationSerializer,
+            [Parameter(Value = typeof(DriverServicePort))] int driverServicePort)
+        {
+            _configurationSerializer = configurationSerializer;
+            Logger.Log(Level.Info, "Binding to driver service at port {0}", driverServicePort);
+            var driverServiceChannel = new Channel("127.0.0.1", driverServicePort, ChannelCredentials.Insecure);
+            _driverServiceStub = new DriverService.DriverServiceClient(driverServiceChannel);
+            Logger.Log(Level.Info, "Channel state {0}", driverServiceChannel.State);
+        }
+
+        public void RegisterDriverClientService(Exception exception)
+        {
+            Logger.Log(Level.Info, "Register driver client error", exception);
+            var registration = new DriverClientRegistration
+            {
+                Exception = GrpcUtils.SerializeException(exception)
+            };
+            _driverServiceStub.RegisterDriverClient(registration);
+        }
+
+        public void RegisterDriverClientService(string host, int port)
+        {
+            Logger.Log(Level.Info, "Register driver client at host {0} port {1}", host, port);
+            var registration = new DriverClientRegistration
+            {
+                Host = host,
+                Port = port
+            };
+            _driverServiceStub.RegisterDriverClient(registration);
+        }
+
+        public void OnShutdown()
+        {
+            Logger.Log(Level.Info, "Driver clean shutdown");
+            _driverServiceStub.Shutdown(new ShutdownRequest());
+        }
+
+        public void OnShutdown(Exception ex)
+        {
+            Logger.Log(Level.Error, "Driver shutdown with error", ex);
+            byte[] errorBytes;
+            try
+            {
+                errorBytes = ByteUtilities.SerializeToBinaryFormat(ex);
+            }
+            catch (SerializationException se)
+            {
+                Logger.Log(Level.Warning, "Unable to serialize exception", ex);
+                errorBytes = ByteUtilities.SerializeToBinaryFormat(
+                    NonSerializableJobException.UnableToSerialize(ex, se));
+            }
+
+            _driverServiceStub.Shutdown(new ShutdownRequest()
+            {
+                Exception = new ExceptionInfo()
+                {
+                    NoError = false,
+                    Message = ex.Message,
+                    Name = ex.Source,
+                    Data = ByteString.CopyFrom(errorBytes)
+                }
+            });
+        }
+
+        public void OnSetAlarm(string alarmId, long timeoutMs)
+        {
+            _driverServiceStub.SetAlarm(new AlarmRequest()
+            {
+                AlarmId = alarmId,
+                TimeoutMs = (int)timeoutMs
+            });
+        }
+
+        public void OnEvaluatorRequest(IEvaluatorRequest evaluatorRequest)
+        {
+            var request = new ResourceRequest()
+            {
+                ResourceCount = evaluatorRequest.Number,
+                Cores = evaluatorRequest.VirtualCore,
+                MemorySize = evaluatorRequest.MemoryMegaBytes,
+                RelaxLocality = evaluatorRequest.RelaxLocality,
+                RuntimeName = evaluatorRequest.RuntimeName,
+                NodeLabel = evaluatorRequest.NodeLabelExpression
+            };
+            if (!string.IsNullOrEmpty(evaluatorRequest.Rack))
+            {
+                request.RackNameList.Add(evaluatorRequest.Rack);
+            }
+            request.NodeNameList.Add(evaluatorRequest.NodeNames);
+            _driverServiceStub.RequestResources(request);
+        }
+
+        public void OnEvaluatorClose(string evalautorId)
+        {
+            _driverServiceStub.AllocatedEvaluatorOp(new AllocatedEvaluatorRequest()
+            {
+                EvaluatorId = evalautorId,
+                CloseEvaluator = true
+            });
+        }
+
+        public void OnEvaluatorSubmit(
+            string evaluatorId,
+            IConfiguration contextConfiguration,
+            Optional<IConfiguration> serviceConfiguration,
+            Optional<IConfiguration> taskConfiguration,
+            List<FileInfo> addFileList, List<FileInfo> addLibraryList)
+        {
+            Logger.Log(Level.Info, "Submitting allocated evaluator {0}", evaluatorId);
+
+            var evaluatorConf =
+                _configurationSerializer.ToString(TangFactory.GetTang().NewConfigurationBuilder().Build());
+            var contextConf = _configurationSerializer.ToString(contextConfiguration);
+            var serviceConf = !serviceConfiguration.IsPresent()
+                ? string.Empty
+                : _configurationSerializer.ToString(serviceConfiguration.Value);
+            var taskConf = !taskConfiguration.IsPresent()
+                ? string.Empty
+                : _configurationSerializer.ToString(taskConfiguration.Value);
+            var request = new AllocatedEvaluatorRequest()
+            {
+                EvaluatorId = evaluatorId,
+                EvaluatorConfiguration = evaluatorConf,
+                ServiceConfiguration = serviceConf,
+                ContextConfiguration = contextConf,
+                TaskConfiguration = taskConf,
+                SetProcess = new AllocatedEvaluatorRequest.Types.EvaluatorProcessRequest()
+                {
+                    ProcessType = AllocatedEvaluatorRequest.Types.EvaluatorProcessRequest.Types.Type.Dotnet
+                }
+            };
+            request.AddFiles.Add(addFileList.Select(f => f.ToString()));
+            request.AddLibraries.Add(addLibraryList.Select(f => f.ToString()));
+            _driverServiceStub.AllocatedEvaluatorOp(request);
+        }
+
+        public void OnContextClose(string contextId)
+        {
+            Logger.Log(Level.Info, "Close context {0}", contextId);
+            _driverServiceStub.ActiveContextOp(new ActiveContextRequest()
+            {
+                ContextId = contextId,
+                CloseContext = true
+            });
+        }
+
+        public void OnContextSubmitContext(string contextId, IConfiguration contextConfiguration)
+        {
+            _driverServiceStub.ActiveContextOp(new ActiveContextRequest()
+            {
+                ContextId = contextId,
+                NewContextRequest = _configurationSerializer.ToString(contextConfiguration)
+            });
+        }
+
+        public void OnContextSubmitTask(string contextId, IConfiguration taskConfiguration)
+        {
+            _driverServiceStub.ActiveContextOp(new ActiveContextRequest()
+            {
+                ContextId = contextId,
+                NewTaskRequest = _configurationSerializer.ToString(taskConfiguration)
+            });
+        }
+
+        public void OnContextMessage(string contextId, byte[] message)
+        {
+            _driverServiceStub.ActiveContextOp(new ActiveContextRequest()
+            {
+                ContextId = contextId,
+                Message = ByteString.CopyFrom(message)
+            });
+        }
+
+        public void OnTaskSuspend(string taskId, Optional<byte[]> message)
+        {
+            var op = new RunningTaskRequest()
+            {
+                TaskId = taskId,
+                Operation = RunningTaskRequest.Types.Operation.Suspend
+            };
+            if (message.IsPresent())
+            {
+                op.Message = ByteString.CopyFrom(message.Value);
+            }
+
+            _driverServiceStub.RunningTaskOp(op);
+        }
+
+        public void OnTaskClose(string taskId, Optional<byte[]> message)
+        {
+            var op = new RunningTaskRequest()
+            {
+                TaskId = taskId,
+                Operation = RunningTaskRequest.Types.Operation.Close
+            };
+            if (message.IsPresent())
+            {
+                op.Message = ByteString.CopyFrom(message.Value);
+            }
+
+            _driverServiceStub.RunningTaskOp(op);
+        }
+
+        public void OnTaskMessage(string taskId, byte[] message)
+        {
+            _driverServiceStub.RunningTaskOp(new RunningTaskRequest()
+            {
+                TaskId = taskId,
+                Message = ByteString.CopyFrom(message),
+                Operation = RunningTaskRequest.Types.Operation.SendMessage
+            });
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverServicePort.cs
similarity index 73%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverServicePort.cs
index 2f3352a..e22e1d7 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/DriverServicePort.cs
@@ -14,13 +14,16 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
 using Org.Apache.REEF.Tang.Annotations;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Driver
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <summary>
+    /// Driver serivce port represents the Java driver server port
+    /// that clients should connect to.
+    /// </summary>
+    [NamedParameter(documentation: "the driver service port")]
+    public sealed class DriverServicePort : Name<int>
     {
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/GRPCUtils.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/GRPCUtils.cs
new file mode 100644
index 0000000..d0efea5
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Driver/GRPCUtils.cs
@@ -0,0 +1,54 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Google.Protobuf;
+using Org.Apache.REEF.Bridge.Core.Proto;
+using Org.Apache.REEF.Common.Exceptions;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.REEF.Bridge.Core.Grpc.Driver
+{
+    internal static class GrpcUtils
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(GrpcUtils));
+
+        public static ExceptionInfo SerializeException(Exception ex)
+        {
+            ByteString serializedException;
+            try
+            {
+                serializedException = ByteString.CopyFrom(ByteUtilities.SerializeToBinaryFormat(ex));
+            }
+            catch (SerializationException se)
+            {
+                Log.Log(Level.Warning, "Unable to serialize exception", ex);
+                serializedException = ByteString.CopyFrom(ByteUtilities.SerializeToBinaryFormat(
+                    NonSerializableJobException.UnableToSerialize(ex, se)));
+            }
+            return new ExceptionInfo()
+            {
+                NoError = false,
+                Name = ex.GetType().ToString(),
+                Message = ex.Message,
+                Data = serializedException
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Org.Apache.REEF.Bridge.Core.Grpc.csproj b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Org.Apache.REEF.Bridge.Core.Grpc.csproj
new file mode 100644
index 0000000..914df3c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Grpc/Org.Apache.REEF.Bridge.Core.Grpc.csproj
@@ -0,0 +1,44 @@
+<Project>
+<!--
+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 (thee
+"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.
+-->
+  <PropertyGroup>
+    <AssemblyName>Org.Apache.REEF.Bridge.Core.Grpc</AssemblyName>
+    <Description>Client for REEF.NET Bridge Core</Description>
+    <PackageTags>REEF Bridge Core Grpc</PackageTags>
+    <!-- Disable default embedded resources as we override this in the project -->
+    <EnableDefaultEmbeddedResourceItems>False</EnableDefaultEmbeddedResourceItems>
+  </PropertyGroup>
+  <Import Project="..\build.Library.props" />
+
+  <ItemGroup>
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Common\Org.Apache.REEF.Bridge.Core.Common.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Proto\Org.Apache.REEF.Bridge.Core.Proto.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Utilities\Org.Apache.REEF.Utilities.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Tang\Org.Apache.REEF.Tang.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Common\Org.Apache.REEF.Common.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Driver\Org.Apache.REEF.Driver.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Network\Org.Apache.REEF.Network.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Wake\Org.Apache.REEF.Wake.csproj" />
+  </ItemGroup>
+  <Import Project="..\build.targets" />
+    
+  <ItemGroup>
+    <EmbeddedResource Include="$(BinDir)$(Configuration)\Org.Apache.REEF.Bridge.JAR\netstandard2.0\*.jar" />
+  </ItemGroup>
+</Project>
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/FailedTask.cs
similarity index 60%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/FailedTask.cs
index 2f3352a..749098a 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/FailedTask.cs
@@ -14,13 +14,31 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
+using Org.Apache.REEF.Common.Tasks;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using System;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.HelloREEF
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <summary>
+    /// A Task that merely prints a greeting and exits.
+    /// </summary>
+    public sealed class FailedTask : ITask
     {
+        [Inject]
+        private FailedTask()
+        {
+        }
+
+        public void Dispose()
+        {
+            Console.WriteLine("Disposed.");
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            throw new IllegalStateException("Hello, Failed REEF Task!");
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloDriver.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloDriver.cs
new file mode 100644
index 0000000..86a43f0
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloDriver.cs
@@ -0,0 +1,133 @@
+// 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.
+
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+
+namespace Org.Apache.REEF.Bridge.Core.HelloREEF
+{
+    /// <summary>
+    /// The Driver for HelloREEF: It requests a single Evaluator and then submits the HelloTask to it.
+    /// </summary>
+    public sealed class HelloDriver :
+        IObserver<IAllocatedEvaluator>,
+        IObserver<IFailedEvaluator>,
+        IObserver<ICompletedEvaluator>,
+        IObserver<IDriverStarted>,
+        IObserver<IDriverStopped>,
+        IObserver<IRunningTask>,
+        IObserver<IActiveContext>,
+        IObserver<ICompletedTask>,
+        IObserver<IFailedTask>
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(HelloDriver));
+        private readonly IEvaluatorRequestor _evaluatorRequestor;
+
+        [Inject]
+        private HelloDriver(IEvaluatorRequestor evaluatorRequestor)
+        {
+            _evaluatorRequestor = evaluatorRequestor;
+        }
+
+        /// <summary>
+        /// Submits the HelloTask to the Evaluator.
+        /// </summary>
+        /// <param name="allocatedEvaluator">Newly allocated evaluator's proxy object.</param>
+        public void OnNext(IAllocatedEvaluator allocatedEvaluator)
+        {
+            Logger.Log(Level.Info, "Evaluator allocated: {0}", allocatedEvaluator);
+
+            var taskConfiguration = TaskConfiguration.ConfigurationModule
+                .Set(TaskConfiguration.Identifier, "HelloTask")
+                .Set(TaskConfiguration.Task, GenericType<HelloTask>.Class)
+                .Build();
+
+            Logger.Log(Level.Verbose, "Submit task: {0}", taskConfiguration);
+            allocatedEvaluator.SubmitTask(taskConfiguration);
+        }
+
+        public void OnNext(IFailedEvaluator value)
+        {
+            Logger.Log(Level.Info, "Failed Evaluator: {0}", value.Id);
+            throw value.EvaluatorException;
+        }
+
+        public void OnNext(ICompletedEvaluator value)
+        {
+            Logger.Log(Level.Info, "Completed Evaluator: {0}", value.Id);
+        }
+
+        public void OnError(Exception error)
+        {
+            throw error;
+        }
+
+        public void OnCompleted()
+        {
+        }
+
+        public void OnNext(IDriverStopped value)
+        {
+            Logger.Log(Level.Info, "HelloDriver stopped at {0}", value.StopTime);
+        }
+
+        /// <summary>
+        /// Called to start the user mode driver.
+        /// </summary>
+        /// <param name="driverStarted">Notification that the Driver is up and running.</param>
+        public void OnNext(IDriverStarted driverStarted)
+        {
+            Logger.Log(Level.Info, "HelloDriver started at {0}", driverStarted.StartTime);
+            _evaluatorRequestor.Submit(_evaluatorRequestor.NewBuilder().SetMegabytes(64).Build());
+        }
+
+        public void OnNext(IRunningTask value)
+        {
+            Logger.Log(Level.Info, "HelloDriver received running task {0}", value.Id);
+        }
+
+        public void OnNext(ICompletedTask value)
+        {
+            Logger.Log(Level.Info, "HelloDriver received completed task {0}", value.Id);
+            var taskConfiguration = TaskConfiguration.ConfigurationModule
+                .Set(TaskConfiguration.Identifier, "FailedTask")
+                .Set(TaskConfiguration.Task, GenericType<FailedTask>.Class)
+                .Build();
+            value.ActiveContext.SubmitTask(taskConfiguration);
+        }
+
+        public void OnNext(IFailedTask value)
+        {
+            Logger.Log(Level.Info, "HelloDriver received failed task {0} with active context {1}", 
+                value.Id, value.GetActiveContext().Value.Id);
+            value.GetActiveContext().Value.Dispose();
+            Logger.Log(Level.Info, "HelloDriver closed active context {0}", value.GetActiveContext().Value.Id);
+        }
+
+        public void OnNext(IActiveContext value)
+        {
+            Logger.Log(Level.Info, "HelloDriver received active context {0}", value.Id);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloREEF.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloREEF.cs
new file mode 100644
index 0000000..4ebd6f1
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloREEF.cs
@@ -0,0 +1,148 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config;
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime;
+using Org.Apache.REEF.Bridge.Core.Grpc.Client;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.REEF.Bridge.Core.HelloREEF
+{
+    /// <summary>
+    /// A Tool that submits HelloREEFDriver for execution.
+    /// </summary>
+    public sealed class HelloREEF
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(HelloREEF));
+
+        private const string Local = "local";
+        private const string YARN = "yarn";
+        private const string HDInsight = "hdi";
+        private const string AzureBatch = "azurebatch";
+
+        [Inject]
+        private HelloREEF()
+        {
+        }
+
+        /// <summary>
+        /// Runs HelloREEF using the IREEFClient passed into the constructor.
+        /// </summary>
+        private void Run(string runtime)
+        {
+            // The driver configuration contains all the needed bindings.
+            var helloDriverConfiguration = DriverApplicationConfiguration.ConfigurationModule
+                .Set(DriverApplicationConfiguration.OnDriverStarted, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnDriverStopped, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnEvaluatorAllocated, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnEvaluatorCompleted, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnEvaluatorFailed, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnContextActive, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskRunning, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskCompleted, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskFailed, GenericType<HelloDriver>.Class)
+                .Set(DriverApplicationConfiguration.CustomTraceLevel, Level.Verbose.ToString())
+                .Build();
+
+            var applicationId = GetApplicationId();
+
+            // The JobSubmission contains the Driver configuration as well as the files needed on the Driver.
+            var driverRuntimeConfiguration = DriverRuntimeConfiguration.ConfigurationModule
+                .Set(DriverRuntimeConfiguration.OsType, GenericType<OsLinux>.Class)
+                .Set(DriverRuntimeConfiguration.JobId, applicationId);
+            driverRuntimeConfiguration = DriverRuntimeConfiguration.AddGlobalAssemblyForType(driverRuntimeConfiguration, typeof(HelloDriver));
+
+            var runtimeConfiguration = GetRuntimeConfiguration(runtime);
+
+            using (var launcher = ClientLauncherFactory.GetLauncher(runtimeConfiguration, driverRuntimeConfiguration.Build()))
+            {
+                var launcherStatus = launcher.SubmitAsync(helloDriverConfiguration).Result;
+                Log.Log(Level.Info, "Final Launch Status {0}", launcherStatus);
+                if (launcherStatus.Error.IsPresent())
+                {
+                    throw launcherStatus.Error.Value;
+                }
+            }
+        }
+
+        private string GetApplicationId()
+        {
+            return "HelloWorldJob-" + DateTime.Now.ToString("ddd-MMM-d-HH-mm-ss-yyyy", CultureInfo.CreateSpecificCulture("en-US"));
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <param name="name"></param>
+        /// <returns></returns>
+        private static IConfiguration GetRuntimeConfiguration(string name)
+        {
+            switch (name)
+            {
+                case Local:
+                    return LocalRuntimeConfiguration.ConfigurationModule
+                        .Set(LocalRuntimeConfiguration.NumberOfEvaluators, "2")
+                        .Build();
+
+                case YARN:
+                    return YarnRuntimeConfiguration.ConfigurationModule.Build();
+
+                case HDInsight:
+                    // To run against HDInsight please replace placeholders below, with actual values for
+                    // connection string, container name (available at Azure portal) and HDInsight
+                    // credentials (username and password)
+                    const string connectionString = "ConnString";
+                    const string continerName = "foo";
+                    return HdInsightRuntimeConfiguration.ConfigurationModule
+                        .Set(HdInsightRuntimeConfiguration.HdInsightPassword, @"pwd")
+                        .Set(HdInsightRuntimeConfiguration.HdInsightUserName, @"foo")
+                        .Set(HdInsightRuntimeConfiguration.HdInsightUrl, @"https://foo.azurehdinsight.net/")
+                        .Build();
+
+                case AzureBatch:
+                    return AzureBatchRuntimeConfiguration.ConfigurationModule
+                        .Set(AzureBatchRuntimeConfiguration.AzureBatchAccountKey, "###")
+                        .Set(AzureBatchRuntimeConfiguration.AzureBatchAccountName, "###")
+                        .Set(AzureBatchRuntimeConfiguration.AzureBatchAccountUri, "###")
+                        .Set(AzureBatchRuntimeConfiguration.AzureBatchPoolId, "###")
+                        .Set(AzureBatchRuntimeConfiguration.AzureStorageAccountKey, "###")
+                        .Set(AzureBatchRuntimeConfiguration.AzureStorageAccountName, "###")
+                        .Set(AzureBatchRuntimeConfiguration.AzureStorageContainerName, "###")
+                        .Build();
+
+                default:
+                    throw new Exception("Unknown runtime: " + name);
+            }
+        }
+
+        public static void MainSimple(string[] args)
+        {
+            var runtime = args.Length > 0 ? args[0] : Local;
+
+            // Execute the HelloREEF, with these parameters injected
+            TangFactory.GetTang()
+                .NewInjector()
+                .GetInstance<HelloREEF>()
+                .Run(runtime);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloTask.cs
similarity index 61%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloTask.cs
index 2f3352a..15aae34 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/HelloTask.cs
@@ -15,12 +15,31 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Org.Apache.REEF.Common.Tasks;
 using Org.Apache.REEF.Tang.Annotations;
+using System;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.HelloREEF
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <summary>
+    /// A Task that merely prints a greeting and exits.
+    /// </summary>
+    public sealed class HelloTask : ITask
     {
+        [Inject]
+        private HelloTask()
+        {
+        }
+
+        public void Dispose()
+        {
+            Console.WriteLine("Disposed.");
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            Console.WriteLine("Hello, REEF!");
+            return null;
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/Org.Apache.REEF.Bridge.Core.HelloREEF.csproj b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/Org.Apache.REEF.Bridge.Core.HelloREEF.csproj
new file mode 100644
index 0000000..6af8bc2
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/Org.Apache.REEF.Bridge.Core.HelloREEF.csproj
@@ -0,0 +1,45 @@
+<Project>
+  <!--
+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.
+-->
+  <PropertyGroup>
+    <AssemblyName>Org.Apache.REEF.Bridge.Core.HelloREEF</AssemblyName>
+    <Description>Hello World Example for REEF.NET Bridge Core</Description>
+  </PropertyGroup>
+  <Import Project="..\build.App.props" />
+
+<ItemGroup>
+    <None Include="$(SolutionDir)App.config">
+      <Link>App.config</Link>
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </None>
+</ItemGroup>
+  <ItemGroup>
+	<ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Grpc\Org.Apache.REEF.Bridge.Core.Grpc.csproj" />
+	<ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Driver\Org.Apache.REEF.Bridge.Core.Driver.csproj" PrivateAssets="none" />
+	<ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Common\Org.Apache.REEF.Bridge.Core.Common.csproj" />
+	<ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Proto\Org.Apache.REEF.Bridge.Core.Proto.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Common\Org.Apache.REEF.Common.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Evaluator\Org.Apache.REEF.Evaluator.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Tang\Org.Apache.REEF.Tang.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Utilities\Org.Apache.REEF.Utilities.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Wake\Org.Apache.REEF.Wake.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.IO\Org.Apache.REEF.IO.csproj" />
+  </ItemGroup>
+  <Import Project="..\build.targets" />
+</Project>
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/Run.cs
similarity index 72%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/Run.cs
index 2f3352a..37043ec 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.HelloREEF/Run.cs
@@ -15,12 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.HelloREEF
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    public sealed class Run
     {
+        /// <summary>
+        /// Program that runs hello reef
+        /// </summary>
+        /// <param name="args"></param>
+        public static void Main(string[] args)
+        {
+            HelloREEF.MainSimple(args);
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Proto/Grpc.targets b/lang/cs/Org.Apache.REEF.Bridge.Core.Proto/Grpc.targets
new file mode 100644
index 0000000..ba3e113
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Proto/Grpc.targets
@@ -0,0 +1,119 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+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.
+-->
+<!-- GRPC CODE GENERATION TARGETS FOR MSBUILD -->
+<Project>
+  <ItemGroup>
+    <PackageReference Include="RoslynCodeTaskFactory" Version="2.0.7" />
+  </ItemGroup>
+  <UsingTask TaskName="GrpcToCSharp" TaskFactory="CodeTaskFactory" AssemblyFile="$(RoslynCodeTaskFactory)" Condition=" '$(RoslynCodeTaskFactory)' != '' ">
+    <ParameterGroup>
+      <!-- The directory where the project is located. This is typically $(ProjectDir) -->
+      <OutputDirectory ParameterType="System.String" Required="true" />
+      <!-- This is the list of C# target files to be generaterd. This is typically @(Compile) -->
+      <SourceFileList ParameterType="System.String" Required="true" />
+      <!-- The directory where the .avsc schema files are located. -->
+      <GrpcSchemaDirectory ParameterType="System.String" Required="true" />
+      <!-- The where gRPC schema generator and dependent libraries are located. -->
+      <GrpcBinaryDirectory ParameterType="System.String" Required="true" />
+      <!-- gRPC tools extension e.g., .exe for windows. -->
+      <GrpcToolsExtension ParameterType="System.String" Required="false" />
+    </ParameterGroup>
+    <Task>
+      <Using Namespace="System" />
+      <Using Namespace="System.IO" />
+      <Using Namespace="System.Diagnostics" />
+      <Using Namespace="System.Threading" />
+      <Using Namespace="System.Threading.Tasks" />
+      <Code Type="Fragment" Language="C#">
+        <![CDATA[
+          // Create a temporary working directory for the log file.
+          string tempDir = Path.GetFullPath(Path.Combine(Path.GetTempPath(), Guid.NewGuid().ToString()));
+          Directory.CreateDirectory(tempDir);
+
+          var grpcCSharpPlugin = "grpc_csharp_plugin";
+          var protocExe = "protoc";
+          if (GrpcToolsExtension != null)
+          {
+            grpcCSharpPlugin += GrpcToolsExtension;
+            protocExe += GrpcToolsExtension;
+          }
+          // Get the full path to the directory where the gRPC .proto files are located.
+          string grpcSrcFileDirectory = Path.GetFullPath(GrpcSchemaDirectory);
+          string grpcToolsDirectory = Path.GetFullPath(GrpcBinaryDirectory);
+          string grpcOutputDirectory = Path.GetFullPath(OutputDirectory);
+          string grpcPluginName = Path.Combine(GrpcBinaryDirectory, grpcCSharpPlugin);
+
+          Directory.CreateDirectory(grpcOutputDirectory);
+           
+          // Get the list of names of .proto files
+          string[] sourceFiles = SourceFileList.Split(';');
+
+          // Setup the conversion process.
+          ProcessStartInfo rProcInfo = new ProcessStartInfo()
+          {
+              RedirectStandardOutput = true,
+              RedirectStandardError = true,
+              UseShellExecute = false,
+              CreateNoWindow = true,
+              FileName = Path.Combine(grpcToolsDirectory, protocExe)
+          };
+
+          bool conversionError = false;
+          using (StreamWriter logWriter = new StreamWriter(Path.Combine(tempDir, "GrpcCodeGeneration.log")))
+          {
+              foreach (string file in sourceFiles)
+              {
+                  logWriter.WriteLine("Processing file: {0}", file);
+                  Console.WriteLine("Processing file: {0}", file);
+				  string fullFileName = Path.Combine(grpcSrcFileDirectory, file);
+                  if (File.Exists(fullFileName))
+                  {
+                    rProcInfo.Arguments = " -I" + grpcSrcFileDirectory + " --csharp_out " + grpcOutputDirectory
+                        + " " + fullFileName + " --grpc_out " + grpcOutputDirectory
+                        + " --plugin=protoc-gen-grpc=" + grpcPluginName;
+                    logWriter.WriteLine("Command: {0}", rProcInfo.Arguments);
+                    StringBuilder stdOutBuilder = new StringBuilder();
+                    StringBuilder stdErrBuilder = new StringBuilder();
+                    using (Process rProc = Process.Start(rProcInfo))
+                    {
+                      var stdOutTask = System.Threading.Tasks.Task.Run(
+                        () => stdOutBuilder.Append(rProc.StandardOutput.ReadToEnd()));
+                      var stdErrTask = System.Threading.Tasks.Task.Run(
+                        () => stdErrBuilder.Append(rProc.StandardError.ReadToEnd()));
+                      rProc.WaitForExit();
+                      conversionError = rProc.ExitCode != 0;
+                      // Wait for std out and error readers.
+                      stdOutTask.Wait();
+                      stdErrTask.Wait();
+                    }
+                    logWriter.WriteLine("[StdOut]:");
+                    logWriter.WriteLine(stdOutBuilder.ToString());
+                    logWriter.WriteLine("[StdErr]:");
+                    logWriter.WriteLine(stdErrBuilder.ToString());
+                  }
+              }
+          }
+          if (!conversionError)
+          {
+            Directory.Delete(tempDir, recursive : true);
+          }
+      ]]>
+      </Code>
+    </Task>
+  </UsingTask>
+</Project>
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Proto/Org.Apache.REEF.Bridge.Core.Proto.csproj b/lang/cs/Org.Apache.REEF.Bridge.Core.Proto/Org.Apache.REEF.Bridge.Core.Proto.csproj
new file mode 100644
index 0000000..e82394b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Proto/Org.Apache.REEF.Bridge.Core.Proto.csproj
@@ -0,0 +1,85 @@
+<!--
+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>
+  <PropertyGroup>
+    <AssemblyName>Org.Apache.REEF.Bridge.Core.Proto</AssemblyName>
+    <Description>REEF Bridge Core Proto</Description>
+    <PackageTags>REEF Bridge Core Proto</PackageTags>
+    <TargetFramework>netstandard2.0</TargetFramework>
+	<ProtoVersion>3.6.0</ProtoVersion>
+	<GrpcVersion>1.13.0</GrpcVersion>
+  </PropertyGroup>
+
+  <Import Project="..\build.props" />
+  <ItemGroup>
+    <PackageReference Include="Google.Protobuf" Version="$(ProtoVersion)" />
+    <PackageReference Include="Grpc.Tools" Version="$(GrpcVersion)" />
+    <PackageReference Include="Grpc.Core" Version="$(GrpcVersion)">
+       <IncludeAssets>all</IncludeAssets>
+	     <ExcludeAssets>none</ExcludeAssets>
+	     <PrivateAssets>none</PrivateAssets>
+	  </PackageReference>
+  </ItemGroup>
+  <PropertyGroup>
+    <GrpcSchemaDirectory>$(REEF_Source_Folder)\lang\common\proto\bridge</GrpcSchemaDirectory>
+	<GrpcOutputDirectory>generated-sources</GrpcOutputDirectory>
+  </PropertyGroup>
+  <!-- Settings for Linux platforms -->
+  <PropertyGroup Condition="'$(OS)' != 'Windows_NT'">
+    <GrpcToolsDirectory>$(PackagesDir)\grpc.tools\$(GrpcVersion)\tools\linux_x64</GrpcToolsDirectory>
+  </PropertyGroup>
+  <!-- Settings for Windows platforms -->
+  <PropertyGroup Condition="'$(OS)' == 'Windows_NT'">
+    <GrpcToolsDirectory>$(PackagesDir)\grpc.tools\$(GrpcVersion)\tools\windows_x64</GrpcToolsDirectory>
+    <GrpcToolsExtension>.exe</GrpcToolsExtension>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="Grpc.Core">
+      <HintPath>$(GRrpcLibraryDirectory)\Grpc.Core.dll</HintPath>
+    </Reference>
+    <Reference Include="Google.Protobuf">
+      <HintPath>$(PackagesDir)\google.protobuf\$(ProtoVersion)\lib\netstandard1.0\Google.Protobuf.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+  </ItemGroup>
+
+  <Import Project="..\build.targets" />
+  
+  <PropertyGroup>  
+    <BuildDependsOn>  
+      CodeGeneration;  
+      $(BuildDependsOn)  
+    </BuildDependsOn>  
+  </PropertyGroup> 
+
+   <Import Project="$(ProjectDir)Grpc.targets" Condition="Exists('$(ProjectDir)Grpc.targets')" />
+  <Target Name="CodeGeneration">
+    <Message Text="Generating C# classes from gRPC proto files in $(GrpcSchemaDirectory) to $(ProjectDir)$(GrpcOutputDirectory) using tools directory $(GrpcToolsDirectory) with extension $(GrpcToolsExtension)" Importance="High" />
+    <GrpcToCSharp OutputDirectory="$(ProjectDir)$(GrpcOutputDirectory)" SourceFileList="DriverCommonProtocol.proto;ClientProtocol.proto;DriverClientProtocol.proto;DriverServiceProtocol.proto" GrpcSchemaDirectory="$(GrpcSchemaDirectory)" GrpcBinaryDirectory="$(GrpcToolsDirectory)" GrpcToolsExtension="$(GrpcToolsExtension)" /> 
+  </Target>
+  
+  <Target Name="UpdateCompileDependencies" AfterTargets="CodeGeneration" BeforeTargets="CoreCompile">
+    <Message Text="Update compile dependencies for gRPC proto files in $(ProjectDir)$(GrpcOutputDirectory)" Importance="High" /> 
+    <ItemGroup>
+      <Compile Include="$(ProjectDir)$(GrpcOutputDirectory)\**" />
+    </ItemGroup>
+  </Target>
+
+
+ </Project>
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/FailDriver.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/FailDriver.cs
new file mode 100644
index 0000000..3d14ae3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/FailDriver.cs
@@ -0,0 +1,406 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Wake.Time;
+using Org.Apache.REEF.Wake.Time.Event;
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Linq;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Driver
+{
+    internal sealed class FailDriver :
+        IObserver<IAllocatedEvaluator>,
+        IObserver<ICompletedEvaluator>,
+        IObserver<IFailedEvaluator>,
+        IObserver<IRunningTask>,
+        IObserver<ICompletedTask>,
+        IObserver<IFailedTask>,
+        IObserver<ITaskMessage>,
+        IObserver<ISuspendedTask>,
+        IObserver<IFailedContext>,
+        IObserver<IClosedContext>,
+        IObserver<IContextMessage>,
+        IObserver<IActiveContext>,
+        IObserver<IDriverStarted>,
+        IObserver<IDriverStopped>,
+        IObserver<Alarm>
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailDriver));
+
+        [NamedParameter("fail message type name")]
+        internal sealed class FailMsgTypeName : Name<string>
+        {
+        }
+
+        private enum DriverState
+        {
+            Init, SendMsg, Suspend, Resume, Close, Failed
+        }
+
+        private static readonly byte[] HelloStringByteArray = ByteUtilities.StringToByteArrays("MESSAGE::HELLO");
+
+        private static readonly int MsgDelay = 1000;
+
+        private static readonly ExpectedMessage[] EventSequence = new ExpectedMessage[]
+        {
+            new ExpectedMessage(typeof(FailDriver), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(IDriverStarted), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(IAllocatedEvaluator), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(IFailedEvaluator), ExpectedMessage.RequiredFlag.Optional),
+            new ExpectedMessage(typeof(IActiveContext), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(IContextMessage), ExpectedMessage.RequiredFlag.Optional),
+            new ExpectedMessage(typeof(IFailedContext), ExpectedMessage.RequiredFlag.Optional),
+            new ExpectedMessage(typeof(IRunningTask), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(Alarm), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(ITaskMessage), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(Alarm), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(ISuspendedTask), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(IRunningTask), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(Alarm), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(IFailedTask), ExpectedMessage.RequiredFlag.Optional),
+            new ExpectedMessage(typeof(ICompletedTask), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(ICompletedTask), ExpectedMessage.RequiredFlag.Optional),
+            new ExpectedMessage(typeof(ICompletedEvaluator), ExpectedMessage.RequiredFlag.Required),
+            new ExpectedMessage(typeof(IDriverStopped), ExpectedMessage.RequiredFlag.Required)
+        };
+
+        private static readonly Dictionary<string, Type> TypeHash = new Dictionary<string, Type>()
+        {
+            {typeof(NoopTask).FullName, typeof(NoopTask)},
+            {typeof(FailDriver).FullName, typeof(FailDriver)},
+            {typeof(IDriverStarted).FullName, typeof(IDriverStarted)},
+            {typeof(IAllocatedEvaluator).FullName, typeof(IAllocatedEvaluator)},
+            {typeof(IFailedEvaluator).FullName, typeof(IFailedEvaluator)},
+            {typeof(IActiveContext).FullName, typeof(IActiveContext)},
+            {typeof(IContextMessage).FullName, typeof(IContextMessage)},
+            {typeof(IFailedContext).FullName, typeof(IFailedContext)},
+            {typeof(IRunningTask).FullName, typeof(IRunningTask)},
+            {typeof(Alarm).FullName, typeof(Alarm)},
+            {typeof(ISuspendedTask).FullName, typeof(ISuspendedTask)},
+            {typeof(IFailedTask).FullName, typeof(IFailedTask)},
+            {typeof(ICompletedTask).FullName, typeof(ICompletedTask)},
+            {typeof(ITaskMessage).FullName, typeof(ITaskMessage)},
+            {typeof(ICompletedEvaluator).FullName, typeof(ICompletedEvaluator)},
+            {typeof(IDriverStopped).FullName, typeof(IDriverStopped)}
+        };
+
+        private readonly Type _failMsgClass;
+        private readonly IEvaluatorRequestor _requestor;
+        private readonly IClock _clock;
+        private IRunningTask _task = null;
+        private int _expectIdx = 0;
+        private DriverState _state = DriverState.Init;
+
+        [Inject]
+        public FailDriver([Parameter(Value = typeof(FailMsgTypeName))] string failMsgTypeName,
+            IEvaluatorRequestor requestor, IClock clock)
+        {
+            _clock = clock;
+            _requestor = requestor;
+            if (TypeHash.ContainsKey(failMsgTypeName))
+            {
+                _failMsgClass = TypeHash[failMsgTypeName];
+            }
+            else
+            {
+                _failMsgClass = typeof(FailDriver).Assembly.GetType(failMsgTypeName, true);
+            }
+            CheckMsgOrder(this);
+        }
+
+        public void OnError(System.Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+
+        private void CheckMsgOrder(object obj)
+        {
+            string msgClassName = _failMsgClass.FullName;
+            Log.Log(Level.Info, "Driver state {0} event sequence {1} message obj type {2}",
+                _state, EventSequence[_expectIdx].Type, obj.GetType());
+
+            if (_state == DriverState.Failed)
+            {
+                // If already failed, do not do anything
+                return;
+            }
+
+            // Simulate failure at this step?
+            if (_failMsgClass.IsInstanceOfType(obj))
+            {
+                _state = DriverState.Failed;
+            }
+
+            // Make sure events arrive in the right order (specified in EVENT_SEQUENCE):
+            bool notFound = true;
+            for (; _expectIdx < EventSequence.Length; ++_expectIdx)
+            {
+                if (EventSequence[_expectIdx].Type.IsInstanceOfType(obj))
+                {
+                    Log.Log(Level.Info, "Object type {0} is instance of expected type {1}", new object[] { obj.GetType(), EventSequence[_expectIdx].Type });
+                    notFound = false;
+                    break;
+                }
+                else if (EventSequence[_expectIdx].Flag == ExpectedMessage.RequiredFlag.Required)
+                {
+                    Log.Log(Level.Info, "Object type {0} is NOT instance of expected type {1}", new object[] { obj.GetType(), EventSequence[_expectIdx].Type });
+                    break;
+                }
+            }
+
+            if (notFound)
+            {
+                Log.Log(Level.Info, "Event out of sequence: Driver state {0} event sequence {1} message obj type {2}",
+                    new object[] { _state, EventSequence[_expectIdx].Type, obj.GetType() });
+                throw new DriverSideFailure("Event out of sequence: " + msgClassName);
+            }
+
+            Log.Log(Level.Info, "{0}: send: {1} got: {2}", new object[] {
+                _state, EventSequence[_expectIdx], msgClassName});
+
+            ++_expectIdx;
+
+            if (_state == DriverState.Failed)
+            {
+                SimulatedDriverFailure ex = new SimulatedDriverFailure(
+                    "Simulated Failure at FailDriver :: " + msgClassName);
+                Log.Log(Level.Info, "Simulated Failure:", ex);
+                throw ex;
+            }
+        }
+
+        private sealed class ExpectedMessage
+        {
+            public enum RequiredFlag
+            {
+                Required,
+                Optional
+            }
+
+            public RequiredFlag Flag { get; }
+
+            public Type Type { get; }
+
+            public ExpectedMessage(Type type, RequiredFlag flag)
+            {
+                Type = type;
+                Flag = flag;
+            }
+        }
+
+        public void OnNext(IAllocatedEvaluator eval)
+        {
+            CheckMsgOrder(eval);
+            try
+            {
+                eval.SubmitContext(ContextConfiguration.ConfigurationModule
+                    .Set(ContextConfiguration.Identifier, "FailContext_" + eval.Id)
+                    .Build());
+            }
+            catch (BindException ex)
+            {
+                Log.Log(Level.Warning, "Context configuration error", ex);
+                throw new IllegalStateException("context configuration error", ex);
+            }
+        }
+
+        public void OnNext(ICompletedEvaluator eval)
+        {
+            CheckMsgOrder(eval);
+            // noop
+        }
+
+        public void OnNext(IFailedEvaluator eval)
+        {
+            Log.Log(Level.Warning, "Evaluator failed: " + eval.Id, eval.EvaluatorException);
+            CheckMsgOrder(eval);
+            throw new IllegalStateException("failed evaluator illegal state", eval.EvaluatorException);
+        }
+
+        public void OnNext(IActiveContext context)
+        {
+            CheckMsgOrder(context);
+            try
+            {
+                context.SubmitTask(TaskConfiguration.ConfigurationModule
+                    .Set(TaskConfiguration.Identifier, "FailTask_" + context.Id)
+                    .Set(TaskConfiguration.Task, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnMessage, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnSuspend, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnClose, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnTaskStop, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnSendMessage, GenericType<NoopTask>.Class)
+                    .Build());
+            }
+            catch (BindException ex)
+            {
+                Log.Log(Level.Warning, "Task configuration error", ex);
+                throw new IllegalStateException("task configuration error", ex);
+            }
+        }
+
+        public void OnNext(IContextMessage message)
+        {
+            CheckMsgOrder(message);
+            // noop
+        }
+
+        public void OnNext(IClosedContext context)
+        {
+            CheckMsgOrder(context);
+            // noop
+        }
+
+        public void OnNext(IFailedContext context)
+        {
+            Log.Log(Level.Warning, "Context failed: " + context.Id);
+            CheckMsgOrder(context);
+        }
+
+        public void OnNext(IRunningTask runningTask)
+        {
+            CheckMsgOrder(runningTask);
+            _task = runningTask;
+            switch (_state)
+            {
+                case DriverState.Init:
+                    Log.Log(Level.Info, "Move to state {0}", DriverState.SendMsg);
+                    _state = DriverState.SendMsg;
+                    break;
+
+                case DriverState.Resume:
+                    Log.Log(Level.Info, "Move to state {0}", DriverState.Close);
+                    _state = DriverState.Close;
+                    break;
+
+                default:
+                    Log.Log(Level.Warning, "Unexpected state at TaskRuntime: {0}", _state);
+                    throw new DriverSideFailure("Unexpected state: " + _state);
+            }
+
+            // After a delay, send message or suspend the task:
+            _clock.ScheduleAlarm(MsgDelay, this);
+        }
+
+        public void OnNext(ISuspendedTask suspendedTask)
+        {
+            CheckMsgOrder(suspendedTask);
+            _state = DriverState.Resume;
+            try
+            {
+                suspendedTask.ActiveContext.SubmitTask(TaskConfiguration.ConfigurationModule
+                    .Set(TaskConfiguration.Identifier, suspendedTask.Id + "_RESUMED")
+                    .Set(TaskConfiguration.Task, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnMessage, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnSuspend, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnClose, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnTaskStop, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.OnSendMessage, GenericType<NoopTask>.Class)
+                    .Set(TaskConfiguration.Memento, ByteUtilities.ByteArraysToString(HelloStringByteArray))
+                    .Build());
+            }
+            catch (BindException ex)
+            {
+                Log.Log(Level.Error, "Task configuration error", ex);
+                throw new DriverSideFailure("Task configuration error", ex);
+            }
+        }
+
+        public void OnNext(ITaskMessage msg)
+        {
+            CheckMsgOrder(msg);
+            Debug.Assert(Enumerable.SequenceEqual(HelloStringByteArray, msg.Message));
+            Debug.Assert(_state == DriverState.SendMsg);
+            _state = DriverState.Suspend;
+            _clock.ScheduleAlarm(MsgDelay, this);
+        }
+
+        public void OnNext(IFailedTask failedTask)
+        {
+            Log.Log(Level.Warning, "Task failed: " + failedTask.Id, failedTask.Reason.OrElse(null));
+            CheckMsgOrder(failedTask);
+            if (failedTask.GetActiveContext().IsPresent())
+            {
+                failedTask.GetActiveContext().Value.Dispose();
+            }
+        }
+
+        public void OnNext(ICompletedTask completedTask)
+        {
+            CheckMsgOrder(completedTask);
+            completedTask.ActiveContext.Dispose();
+        }
+
+        public void OnNext(IDriverStarted time)
+        {
+            CheckMsgOrder(time);
+            _requestor.Submit(_requestor.NewBuilder()
+                .SetNumber(1).SetMegabytes(128).SetCores(1).Build());
+        }
+
+        public void OnNext(Alarm time)
+        {
+            CheckMsgOrder(time);
+            switch (_state)
+            {
+                case DriverState.SendMsg:
+                    Log.Log(Level.Info, "Send message to task {0}", _task.Id);
+                    _task.Send(HelloStringByteArray);
+                    break;
+
+                case DriverState.Suspend:
+                    Log.Log(Level.Info, "Suspend task {0}", _task.Id);
+                    _task.Suspend();
+                    break;
+
+                case DriverState.Close:
+                    Log.Log(Level.Info, "Close task {0}", _task.Id);
+                    _task.Dispose();
+                    break;
+
+                default:
+                    Log.Log(Level.Warning, "Unexpected state at AlarmHandler: {0}", _state);
+                    throw new DriverSideFailure("Unexpected state: " + _state);
+            }
+        }
+
+        public void OnNext(IDriverStopped time)
+        {
+            CheckMsgOrder(time);
+            // noop
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/FailDriverClient.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/FailDriverClient.cs
new file mode 100644
index 0000000..99b54e3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/FailDriverClient.cs
@@ -0,0 +1,94 @@
+// 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.
+
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config;
+using Org.Apache.REEF.Bridge.Core.Grpc.Client;
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using System;
+using System.Threading;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Driver
+{
+    public sealed class FailDriverClient
+    {
+        private static IConfiguration BuildAppDriverConfig(Type failMsgClass)
+        {
+            IConfiguration driverConfig = DriverApplicationConfiguration.ConfigurationModule
+                .Set(DriverApplicationConfiguration.OnDriverStarted, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnDriverStopped, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnEvaluatorAllocated, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnEvaluatorCompleted, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnEvaluatorFailed, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnContextActive, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnContextMessage, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnContextClosed, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnContextFailed, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskRunning, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskSuspended, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskMessage, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskFailed, GenericType<FailDriver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskCompleted, GenericType<FailDriver>.Class)
+                .Build();
+
+            return TangFactory.GetTang().NewConfigurationBuilder(driverConfig)
+                .BindNamedParameter(typeof(FailDriver.FailMsgTypeName), failMsgClass.FullName)
+                .Build();
+        }
+
+        /**
+         * Run REEF on specified runtime and fail (raise an exception) in a specified class.
+         * @param failMsgClass A class that should fail during the test.
+         * @param runtimeConfig REEF runtime configuration. Can be e.g. Local or YARN.
+         * @param timeOut REEF application timeout.
+         * @return launcher status - usually FAIL.
+         * @throws InjectionException configuration error.
+         */
+
+        public static LauncherStatus RunClient(Type failMsgClass, IConfiguration runtimeConfig, TimeSpan timeout)
+        {
+            // The JobSubmission contains the Driver configuration as well as the files needed on the Driver.
+            var driverRuntimeConfiguration = DriverRuntimeConfiguration.ConfigurationModule
+                .Set(DriverRuntimeConfiguration.OsType, GenericType<OsWindows>.Class)
+                .Set(DriverRuntimeConfiguration.JobId, "Fail_" + failMsgClass.Name);
+            driverRuntimeConfiguration = DriverRuntimeConfiguration
+                .AddGlobalAssemblyForType(driverRuntimeConfiguration, typeof(FailDriver));
+
+            var source = new CancellationTokenSource();
+            source.CancelAfter(timeout);
+            using (var launcher = ClientLauncherFactory.GetLauncher(runtimeConfig, driverRuntimeConfiguration.Build()))
+            {
+                var task = launcher.SubmitAsync(BuildAppDriverConfig(failMsgClass), source.Token);
+                try
+                {
+                    return task.Result;
+                }
+                catch (Exception)
+                {
+                    if (task.IsCanceled)
+                    {
+                        throw new TimeoutException($"Job timed out after {timeout}");
+                    }
+
+                    throw;
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/NoopTask.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/NoopTask.cs
new file mode 100644
index 0000000..1e5fd48
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Driver/NoopTask.cs
@@ -0,0 +1,151 @@
+// 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.
+
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Common.Tasks.Events;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Threading;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Driver
+{
+    internal sealed class NoopTask :
+        ITask,
+        ITaskMessageSource,
+        IDriverMessageHandler,
+        IObserver<ISuspendEvent>,
+        IObserver<ITaskStop>,
+        IObserver<ICloseEvent>
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(NoopTask));
+
+        private static readonly TaskMessage InitMessage =
+            TaskMessage.From("nooptask", ByteUtilities.StringToByteArrays("MESSAGE::INIT"));
+
+        private bool _isRunning = true;
+        private Optional<TaskMessage> _message = Utilities.Optional<TaskMessage>.Empty();
+
+        [Inject]
+        private NoopTask()
+        {
+            Log.Log(Level.Info, "NoopTask created.");
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            _isRunning = true;
+            while (_isRunning)
+            {
+                try
+                {
+                    Log.Log(Level.Info, "NoopTask.call(): Waiting for the message.");
+                    lock (this)
+                    {
+                        Monitor.Wait(this);
+                    }
+                }
+                catch (System.Threading.ThreadInterruptedException ex)
+                {
+                    Log.Log(Level.Warning, "NoopTask.wait() interrupted.", ex);
+                }
+            }
+
+            Log.Log(Level.Info,
+                "NoopTask.call(): Exiting with message {0}",
+                ByteUtilities.ByteArraysToString(_message.OrElse(InitMessage).Message));
+            return _message.OrElse(InitMessage).Message;
+        }
+
+        public Optional<TaskMessage> Message
+        {
+            get
+            {
+                Log.Log(Level.Info,
+                    "NoopTask.getMessage() invoked: {0}",
+                    ByteUtilities.ByteArraysToString(_message.OrElse(InitMessage).Message));
+                return _message;
+            }
+        }
+
+        public void Dispose()
+        {
+            Log.Log(Level.Info, "NoopTask.stopTask() invoked.");
+            _isRunning = false;
+            lock (this)
+            {
+                Monitor.Pulse(this);
+            }
+        }
+
+        public void OnError(System.Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+
+        /// <summary>
+        /// Handler for SuspendEvent.
+        /// </summary>
+        /// <param name="suspendEvent"></param>
+        public void OnNext(ISuspendEvent suspendEvent)
+        {
+            Log.Log(Level.Info, "NoopTask.TaskSuspendHandler.OnNext() invoked.");
+            Dispose();
+        }
+
+        /// <summary>
+        /// Handler for TaskStop.
+        /// </summary>
+        /// <param name="value"></param>
+        public void OnNext(ITaskStop value)
+        {
+            Log.Log(Level.Info, "NoopTask.TaskStopHandler.OnNext() invoked.");
+        }
+
+        /// <summary>
+        /// Handler for CloseEvent.
+        /// </summary>
+        /// <param name="closeEvent"></param>
+        public void OnNext(ICloseEvent closeEvent)
+        {
+            Log.Log(Level.Info, "NoopTask.TaskCloseHandler.OnNext() invoked.");
+            Dispose();
+        }
+
+        /// <summary>
+        /// Handler for DriverMessage.
+        /// </summary>
+        /// <param name="driverMessage"></param>
+        public void Handle(IDriverMessage driverMessage)
+        {
+            byte[] msg = driverMessage.Message.Value;
+            Log.Log(Level.Info,
+                "NoopTask.DriverMessageHandler.Handle() invoked: {0}",
+                ByteUtilities.ByteArraysToString(msg));
+            lock (this)
+            {
+                _message = Utilities.Optional<TaskMessage>.Of(TaskMessage.From("nooptask", msg));
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/DriverSideFailure.cs
similarity index 59%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/DriverSideFailure.cs
index 2f3352a..2681b27 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/DriverSideFailure.cs
@@ -15,12 +15,24 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using System;
+using System.Runtime.Serialization;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    [Serializable]
+    public sealed class DriverSideFailure : System.Exception
     {
+        public DriverSideFailure(string message) : base(message)
+        {
+        }
+
+        public DriverSideFailure(string message, System.Exception innerException) : base(message, innerException)
+        {
+        }
+
+        protected DriverSideFailure(SerializationInfo info, StreamingContext context) : base(info, context)
+        {
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/SimulatedDriverFailure.cs
similarity index 58%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/SimulatedDriverFailure.cs
index 2f3352a..cdf0fc0 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/SimulatedDriverFailure.cs
@@ -15,12 +15,24 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using Org.Apache.REEF.Tang.Annotations;
+using System;
+using System.Runtime.Serialization;
 
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    [Serializable]
+    public sealed class SimulatedDriverFailure : System.Exception
     {
+        public SimulatedDriverFailure(string message) : base(message)
+        {
+        }
+
+        public SimulatedDriverFailure(string message, System.Exception innerException) : base(message, innerException)
+        {
+        }
+
+        protected SimulatedDriverFailure(SerializationInfo info, StreamingContext context) : base(info, context)
+        {
+        }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/SimulatedTaskFailure.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/SimulatedTaskFailure.cs
new file mode 100644
index 0000000..e1c0495
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Exception/SimulatedTaskFailure.cs
@@ -0,0 +1,37 @@
+// 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.
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException
+{
+    [Serializable]
+    internal sealed class SimulatedTaskFailure : System.Exception
+    {
+        public SimulatedTaskFailure(string message) : base(message)
+        {
+        }
+
+        public SimulatedTaskFailure(string message, System.Exception innerException) : base(message, innerException)
+        {
+        }
+
+        protected SimulatedTaskFailure(SerializationInfo info, StreamingContext context) : base(info, context)
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/BlockingCallTaskBase.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/BlockingCallTaskBase.cs
new file mode 100644
index 0000000..bc2b350
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/BlockingCallTaskBase.cs
@@ -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.
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.Threading;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
+{
+    internal abstract class BlockingCallTaskBase : ITask
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(BlockingCallTaskBase));
+
+        private bool _isRunning = true;
+
+        public void Dispose()
+        {
+            lock (this)
+            {
+                _isRunning = false;
+                Monitor.Pulse(this);
+            }
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            lock (this)
+            {
+                Log.Log(Level.Info, "BlockingCallTaskBase.call() invoked. Waiting for the message.");
+                while (_isRunning)
+                {
+                    try
+                    {
+                        Monitor.Wait(this);
+                    }
+                    catch (System.Exception ex)
+                    {
+                        Log.Log(Level.Warning, "wait() interrupted.", ex);
+                    }
+                }
+            }
+            return new byte[0];
+        }
+
+        public void OnError(System.Exception error)
+        {
+            Log.Log(Level.Error, "OnError called", error);
+            throw error;
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotSupportedException("OnComplete not supported by task");
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/Driver.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/Driver.cs
new file mode 100644
index 0000000..4ace9a1
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/Driver.cs
@@ -0,0 +1,185 @@
+// 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.
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
+{
+    internal sealed class Driver :
+        IObserver<IAllocatedEvaluator>,
+        IObserver<IRunningTask>,
+        IObserver<IActiveContext>,
+        IObserver<IDriverStarted>
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(Driver));
+
+        [NamedParameter("the fail task to test")]
+        internal sealed class FailTaskName : Name<string>
+        {
+        }
+
+        private readonly string _failTaskName;
+
+        private readonly IEvaluatorRequestor _evaluatorRequestor;
+
+        private string _taskId;
+
+        [Inject]
+        private Driver(IEvaluatorRequestor evaluatorRequestor,
+            [Parameter(Value = typeof(FailTaskName))]
+            string failTaskName)
+        {
+            _failTaskName = failTaskName;
+            _evaluatorRequestor = evaluatorRequestor;
+        }
+
+        public void OnError(System.Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnNext(IAllocatedEvaluator eval)
+        {
+            try
+            {
+                _taskId = _failTaskName + "_" + eval.Id;
+                Log.Log(Level.Info, "Submit task: {0}", _taskId);
+
+                IConfiguration contextConfig =
+                    ContextConfiguration.ConfigurationModule.Set(ContextConfiguration.Identifier, _taskId).Build();
+
+                ConfigurationModule taskConfig =
+                    TaskConfiguration.ConfigurationModule.Set(TaskConfiguration.Identifier, _taskId);
+
+                switch (_failTaskName)
+                {
+                    case "FailTask":
+                        taskConfig = taskConfig.Set(TaskConfiguration.Task, GenericType<FailTask>.Class);
+                        break;
+
+                    case "FailTaskCall":
+                        taskConfig = taskConfig.Set(TaskConfiguration.Task, GenericType<FailTaskCall>.Class);
+                        break;
+
+                    case "FailTaskMsg":
+                        taskConfig = taskConfig
+                            .Set(TaskConfiguration.Task, GenericType<FailTaskMsg>.Class)
+                            .Set(TaskConfiguration.OnMessage, GenericType<FailTaskMsg>.Class);
+                        break;
+
+                    case "FailTaskSuspend":
+                        taskConfig = taskConfig
+                            .Set(TaskConfiguration.Task, GenericType<FailTaskSuspend>.Class)
+                            .Set(TaskConfiguration.OnSuspend, GenericType<FailTaskSuspend>.Class);
+                        break;
+
+                    case "FailTaskStart":
+                        taskConfig = taskConfig
+                            .Set(TaskConfiguration.Task, GenericType<FailTaskStart>.Class)
+                            .Set(TaskConfiguration.OnTaskStart, GenericType<FailTaskStart>.Class);
+                        break;
+
+                    case "FailTaskStop":
+                        taskConfig = taskConfig
+                            .Set(TaskConfiguration.Task, GenericType<FailTaskStop>.Class)
+                            .Set(TaskConfiguration.OnTaskStop, GenericType<FailTaskStop>.Class)
+                            .Set(TaskConfiguration.OnClose, GenericType<FailTaskStop>.Class);
+                        break;
+
+                    case "FailTaskClose":
+                        taskConfig = taskConfig
+                            .Set(TaskConfiguration.Task, GenericType<FailTaskClose>.Class)
+                            .Set(TaskConfiguration.OnClose, GenericType<FailTaskClose>.Class);
+                        break;
+
+                    default:
+                        break;
+                }
+
+                eval.SubmitContextAndTask(contextConfig, taskConfig.Build());
+            }
+            catch (BindException ex)
+            {
+                Log.Log(Level.Warning, "Configuration error", ex);
+                throw new DriverSideFailure("Configuration error", ex);
+            }
+        }
+
+        public void OnNext(IRunningTask task)
+        {
+            Log.Log(Level.Info,
+                "TaskRuntime: {0} expect {1}",
+                task.Id, _taskId);
+
+            if (!_taskId.Equals(task.Id))
+            {
+                throw new DriverSideFailure($"Task ID {task.Id} not equal expected ID {_taskId}");
+            }
+
+            switch (_failTaskName)
+            {
+                case "FailTaskMsg":
+                    Log.Log(Level.Info, "TaskRuntime: Send message: {0}", task);
+                    task.Send(new byte[0]);
+                    break;
+
+                case "FailTaskSuspend":
+                    Log.Log(Level.Info, "TaskRuntime: Suspend: {0}", task);
+                    task.Suspend();
+                    break;
+
+                case "FailTaskStop":
+                case "FailTaskClose":
+                    Log.Log(Level.Info, "TaskRuntime: Stop/Close: {0}", task);
+                    task.Dispose();
+                    break;
+
+                default:
+                    break;
+            }
+        }
+
+        public void OnNext(IActiveContext context)
+        {
+            throw new DriverSideFailure($"Unexpected ActiveContext message: {context.Id}");
+        }
+
+        public void OnNext(IDriverStarted start)
+        {
+            Log.Log(Level.Info, "StartTime: {0}", start);
+            _evaluatorRequestor.Submit(_evaluatorRequestor.NewBuilder()
+                .SetNumber(1).SetMegabytes(128).SetNumber(1).Build());
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTask.cs
similarity index 55%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTask.cs
index 29a7d8e..6f8f296 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTask.cs
@@ -14,37 +14,33 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Tasks;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using Org.Apache.REEF.Utilities.Logging;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal class FailTask : ITask
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
-        {
-        }
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailTask));
 
-        public void OnNext(IFailedTask value)
+        [Inject]
+        private FailTask()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            SimulatedTaskFailure ex = new SimulatedTaskFailure("FailTask constructor called.");
+            Log.Log(Level.Info, "FailTask created - failing now: {0}", ex);
+            throw ex;
         }
 
-        public void OnError(Exception error)
+        public void Dispose()
         {
-            throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public byte[] Call(byte[] memento)
         {
-            throw new NotImplementedException();
+            throw new IllegalStateException("should not make it to this point");
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskCall.cs
similarity index 55%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskCall.cs
index 29a7d8e..b4f37e9 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskCall.cs
@@ -14,37 +14,32 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Tasks;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class FailTaskCall : ITask
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
-        {
-        }
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailTaskCall));
 
-        public void OnNext(IFailedTask value)
+        [Inject]
+        private FailTaskCall()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            Log.Log(Level.Info, "task created");
         }
 
-        public void OnError(Exception error)
+        public void Dispose()
         {
-            throw new NotImplementedException();
         }
 
-        public void OnCompleted()
+        public byte[] Call(byte[] memento)
         {
-            throw new NotImplementedException();
+            SimulatedTaskFailure ex = new SimulatedTaskFailure("FailTaskCall.call() invoked.");
+            Log.Log(Level.Info, "FailTaskCall.call() invoked: {0}", ex);
+            throw ex;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskClient.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskClient.cs
new file mode 100644
index 0000000..5bd5c89
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskClient.cs
@@ -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.
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config;
+using Org.Apache.REEF.Bridge.Core.Grpc.Client;
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using System;
+using System.Threading;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
+{
+    internal static class FailTaskClient
+    {
+        private static IConfiguration BuildAppDriverConfig(Type failTaskName)
+        {
+            IConfiguration driverConfig = DriverApplicationConfiguration.ConfigurationModule
+                .Set(DriverApplicationConfiguration.OnDriverStarted, GenericType<Driver>.Class)
+                .Set(DriverApplicationConfiguration.OnEvaluatorAllocated, GenericType<Driver>.Class)
+                .Set(DriverApplicationConfiguration.OnContextActive, GenericType<Driver>.Class)
+                .Set(DriverApplicationConfiguration.OnTaskRunning, GenericType<Driver>.Class)
+                .Build();
+
+            return TangFactory.GetTang().NewConfigurationBuilder(driverConfig)
+                .BindNamedParameter(typeof(Driver.FailTaskName), failTaskName.Name)
+                .Build();
+        }
+
+        /**
+         * Run REEF on specified runtime and fail (raise an exception) in a specified class.
+         * @param failTaskType A class that should fail during the test.
+         * @param runtimeConfig REEF runtime configuration. Can be e.g. Local or YARN.
+         * @param timeOut REEF application timeout.
+         * @return launcher status - usually FAIL.
+         * @throws InjectionException configuration error.
+         */
+
+        public static LauncherStatus Run(Type failTaskType, IConfiguration runtimeConfig, TimeSpan timeout)
+        {
+            var driverRuntimeConfiguration = DriverRuntimeConfiguration.ConfigurationModule
+                .Set(DriverRuntimeConfiguration.OsType, GenericType<OsWindows>.Class)
+                .Set(DriverRuntimeConfiguration.JobId, "Fail_" + failTaskType.Name);
+            driverRuntimeConfiguration = DriverRuntimeConfiguration
+                .AddGlobalAssemblyForType(driverRuntimeConfiguration, typeof(Driver));
+
+            var source = new CancellationTokenSource();
+            source.CancelAfter(timeout);
+            using (var launcher = ClientLauncherFactory.GetLauncher(runtimeConfig, driverRuntimeConfiguration.Build()))
+            {
+                var task = launcher.SubmitAsync(BuildAppDriverConfig(failTaskType), source.Token);
+                try
+                {
+                    return task.Result;
+                }
+                catch (Exception)
+                {
+                    if (task.IsCanceled)
+                    {
+                        throw new TimeoutException($"Job timed out after {timeout}");
+                    }
+
+                    throw;
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskClose.cs
similarity index 54%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskClose.cs
index 29a7d8e..caf7290 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskClose.cs
@@ -14,37 +14,29 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Tasks.Events;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class FailTaskClose : BlockingCallTaskBase, IObserver<ICloseEvent>
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
-        {
-        }
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailTaskClose));
 
-        public void OnNext(IFailedTask value)
-        {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
-        }
-
-        public void OnError(Exception error)
+        [Inject]
+        private FailTaskClose()
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Info, "task created");
         }
 
-        public void OnCompleted()
+        public void OnNext(ICloseEvent value)
         {
-            throw new NotImplementedException();
+            SimulatedTaskFailure ex = new SimulatedTaskFailure("FailTaskClose.send() invoked.");
+            Log.Log(Level.Info, "FailTaskClose.onNext() invoked. Raise exception: {0}", ex);
+            throw ex;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskMsg.cs
similarity index 54%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskMsg.cs
index 29a7d8e..1a9ab47 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskMsg.cs
@@ -14,37 +14,29 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Common.Tasks.Events;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class FailTaskMsg : BlockingCallTaskBase, IDriverMessageHandler
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
-        {
-        }
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailTaskMsg));
 
-        public void OnNext(IFailedTask value)
-        {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
-        }
-
-        public void OnError(Exception error)
+        [Inject]
+        private FailTaskMsg()
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Info, "task created");
         }
 
-        public void OnCompleted()
+        public void Handle(IDriverMessage message)
         {
-            throw new NotImplementedException();
+            SimulatedTaskFailure ex = new SimulatedTaskFailure("FailTaskMsg.send() invoked.");
+            Log.Log(Level.Info, "FailTaskMsg.send() invoked: {0}", ex);
+            throw ex;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskStart.cs
similarity index 54%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskStart.cs
index 29a7d8e..f62195a 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskStart.cs
@@ -14,37 +14,29 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Tasks.Events;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class FailTaskStart : BlockingCallTaskBase, IObserver<ITaskStart>
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
-        {
-        }
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailTaskStart));
 
-        public void OnNext(IFailedTask value)
-        {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
-        }
-
-        public void OnError(Exception error)
+        [Inject]
+        private FailTaskStart()
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Info, "task created");
         }
 
-        public void OnCompleted()
+        public void OnNext(ITaskStart value)
         {
-            throw new NotImplementedException();
+            SimulatedTaskFailure ex = new SimulatedTaskFailure("FailTaskMsg.send() invoked.");
+            Log.Log(Level.Info, "FailTaskMsg.send() invoked: {0}", ex);
+            throw ex;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskStop.cs
similarity index 54%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskStop.cs
index 29a7d8e..4571af3 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskStop.cs
@@ -14,37 +14,34 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Tasks.Events;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class FailTaskStop : BlockingCallTaskBase, IObserver<ITaskStop>, IObserver<ICloseEvent>
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
-        {
-        }
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailTaskStop));
 
-        public void OnNext(IFailedTask value)
+        [Inject]
+        private FailTaskStop()
         {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
+            Log.Log(Level.Info, "task created");
         }
 
-        public void OnError(Exception error)
+        public void OnNext(ICloseEvent value)
         {
-            throw new NotImplementedException();
+            Dispose();
         }
 
-        public void OnCompleted()
+        public void OnNext(ITaskStop value)
         {
-            throw new NotImplementedException();
+            SimulatedTaskFailure ex = new SimulatedTaskFailure("FailTaskStop Failure.");
+            Log.Log(Level.Info, "FailTaskStop.onNext() invoked. Raise exception: {0}", ex);
+            throw ex;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskSuspend.cs
similarity index 54%
copy from lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskSuspend.cs
index 29a7d8e..e11dbd0 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultTaskFailureHandler.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Task/FailTaskSuspend.cs
@@ -14,37 +14,29 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-using System;
-using System.Globalization;
-using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Tasks.Events;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
 
-namespace Org.Apache.REEF.Driver.Defaults
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Task
 {
-    /// <summary>
-    /// Default event handler used for FailedTask: It crashes the driver.
-    /// </summary>
-    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    internal sealed class FailTaskSuspend : BlockingCallTaskBase, IObserver<ISuspendEvent>
     {
-        [Inject]
-        public DefaultTaskFailureHandler()
-        {
-        }
+        private static readonly Logger Log = Logger.GetLogger(typeof(FailTaskSuspend));
 
-        public void OnNext(IFailedTask value)
-        {
-            throw new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Task {0} has failed, and no handler was bound for IFailedTask", value.Id));
-        }
-
-        public void OnError(Exception error)
+        [Inject]
+        private FailTaskSuspend()
         {
-            throw new NotImplementedException();
+            Log.Log(Level.Info, "task created");
         }
 
-        public void OnCompleted()
+        public void OnNext(ISuspendEvent value)
         {
-            throw new NotImplementedException();
+            SimulatedTaskFailure ex = new SimulatedTaskFailure("FailTaskSuspend.send() invoked.");
+            Log.Log(Level.Info, "FailTaskSuspend.send() invoked: {0}", ex);
+            throw ex;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/FailDriverTest.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/FailDriverTest.cs
new file mode 100644
index 0000000..96ee0db
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/FailDriverTest.cs
@@ -0,0 +1,116 @@
+// 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.
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.Driver;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Wake.Time.Event;
+using System;
+using Xunit;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Test
+{
+    public sealed class FailDriverTest
+    {
+        private static readonly TimeSpan JobTimeout = TimeSpan.FromMinutes(5);
+
+        private static void FailOn(Type type)
+        {
+            var launchStatus = FailDriverClient.RunClient(type, LocalRuntimeConfiguration.ConfigurationModule.Build(), JobTimeout);
+            TestUtils.AssertLauncherFailure(launchStatus, typeof(SimulatedDriverFailure));
+        }
+
+        [Fact]
+        public void TestFailDriverConstructor()
+        {
+            FailOn(typeof(FailDriver));
+        }
+
+        [Fact]
+        public void TestFailDriverStart()
+        {
+            FailOn(typeof(IDriverStarted));
+        }
+
+        [Fact]
+        public void TestFailDriverAllocatedEvaluator()
+        {
+            FailOn(typeof(IAllocatedEvaluator));
+        }
+
+        [Fact]
+        public void TestFailDriverActiveContext()
+        {
+            FailOn(typeof(IActiveContext));
+        }
+
+        [Fact]
+        public void TestFailDriverRunningTask()
+        {
+            FailOn(typeof(IRunningTask));
+        }
+
+        [Fact]
+        public void TestFailDriverTaskMessage()
+        {
+            FailOn(typeof(ITaskMessage));
+        }
+
+        [Fact]
+        public void TestFailDriverSuspendedTask()
+        {
+            FailOn(typeof(ISuspendedTask));
+        }
+
+        [Fact]
+        public void TestFailDriverCompletedTask()
+        {
+            FailOn(typeof(ICompletedTask));
+        }
+
+        [Fact]
+        public void TestFailDriverCompletedEvaluator()
+        {
+            FailOn(typeof(ICompletedEvaluator));
+        }
+
+        [Fact]
+        public void TestFailDriverAlarm()
+        {
+            FailOn(typeof(Alarm));
+        }
+
+        [Fact]
+        public void TestFailDriverStop()
+        {
+            FailOn(typeof(IDriverStopped));
+        }
+
+        [Fact]
+        public void TestDriverCompleted()
+        {
+            // NoopTask can be replaced with any other class never used in FailDriver
+            LauncherStatus status = FailDriverClient.RunClient(typeof(NoopTask),
+                LocalRuntimeConfiguration.ConfigurationModule.Build(), JobTimeout);
+            Assert.Equal(LauncherStatus.CompletedStatus, status);
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/FailTaskTest.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/FailTaskTest.cs
new file mode 100644
index 0000000..127ad6b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/FailTaskTest.cs
@@ -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.
+using Org.Apache.REEF.Bridge.Core.Common.Client.Config.Runtime;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.Task;
+using Org.Apache.REEF.Bridge.Core.Tests.Fail.ThreadInterruptedException;
+using System;
+using Xunit;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Test
+{
+    public sealed class FailTaskTest
+    {
+        private static readonly TimeSpan JobTimeout = TimeSpan.FromMinutes(5);
+
+        private static void FailOn(Type type)
+        {
+            var launchStatus = FailTaskClient.Run(type, LocalRuntimeConfiguration.ConfigurationModule.Build(), JobTimeout);
+            TestUtils.AssertLauncherFailure(launchStatus, typeof(SimulatedTaskFailure));
+        }
+
+        [Fact]
+        public void TestFailTask()
+        {
+            FailOn(typeof(FailTask));
+        }
+
+        [Fact]
+        public void TestFailTaskCall()
+        {
+            FailOn(typeof(FailTaskCall));
+        }
+
+        [Fact]
+        public void TestFailTaskMsg()
+        {
+            FailOn(typeof(FailTaskMsg));
+        }
+
+        [Fact]
+        public void TestFailTaskSuspend()
+        {
+            FailOn(typeof(FailTaskSuspend));
+        }
+
+        [Fact]
+        public void TestFailTaskStart()
+        {
+            FailOn(typeof(FailTaskStart));
+        }
+
+        [Fact]
+        public void TestFailTaskStop()
+        {
+            FailOn(typeof(FailTaskStop));
+        }
+
+        [Fact]
+        public void TestFailTaskClose()
+        {
+            FailOn(typeof(FailTaskClose));
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/TestUtils.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/TestUtils.cs
new file mode 100644
index 0000000..ae41a7d
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Fail/Test/TestUtils.cs
@@ -0,0 +1,49 @@
+// 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.
+using Org.Apache.REEF.Common.Client;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using Xunit;
+
+namespace Org.Apache.REEF.Bridge.Core.Tests.Fail.Test
+{
+    internal static class TestUtils
+    {
+        private static readonly Logger Log = Logger.GetLogger(typeof(TestUtils));
+
+        public static void AssertLauncherFailure(LauncherStatus status, Type expectedFailureType)
+        {
+            Assert.Equal(LauncherStatus.FailedStatus, status);
+            var ex = status.Error.OrElse(null);
+            Assert.True(HasCause(ex, expectedFailureType), "Unexpected error: " + ex);
+        }
+
+        private static bool HasCause(Exception ex, Type exceptionType)
+        {
+            for (var exception = ex; exception != null; exception = exception.InnerException)
+            {
+                Log.Log(Level.Info, "Exception type {0}", exception.GetType());
+                if (exceptionType.IsInstanceOfType(exception))
+                {
+                    return true;
+                }
+                Log.Log(Level.Info, "Not instance of {0}", exceptionType);
+            }
+            return false;
+        }
+    }
+}
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Org.Apache.REEF.Bridge.Core.Tests.csproj b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Org.Apache.REEF.Bridge.Core.Tests.csproj
new file mode 100644
index 0000000..4f285a6
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Org.Apache.REEF.Bridge.Core.Tests.csproj
@@ -0,0 +1,33 @@
+<Project>
+  <!--
+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.
+-->
+  <PropertyGroup>
+    <AssemblyName>Org.Apache.REEF.Bridge.Core.Tests</AssemblyName>
+  </PropertyGroup>
+  <Import Project="..\build.Test.props" />
+  <ItemGroup>
+    <PackageReference Include="System.Threading.Threadpool" Version="4.3.0" />
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Common\Org.Apache.REEF.Bridge.Core.Common.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Grpc\Org.Apache.REEF.Bridge.Core.Grpc.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Driver\Org.Apache.REEF.Bridge.Core.Driver.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Bridge.Core.Proto\Org.Apache.REEF.Bridge.Core.Proto.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Common\Org.Apache.REEF.Common.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Evaluator\Org.Apache.REEF.Evaluator.csproj" />
+    <ProjectReference Include="..\Org.Apache.REEF.Tang\Org.Apache.REEF.Tang.csproj" />
+  </ItemGroup>
+  <Import Project="..\build.Test.targets" />
+</Project>
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Properties/AssemblyInfo.cs
similarity index 84%
copy from lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs
copy to lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Properties/AssemblyInfo.cs
index e7b4768..881078a 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.Core.Tests/Properties/AssemblyInfo.cs
@@ -15,8 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System.Runtime.CompilerServices;
-
-// Allow the bridge access to `internal` APIs
-[assembly: InternalsVisibleTo("Org.Apache.REEF.Bridge")]
+using Xunit;
 
+[assembly: CollectionBehavior(DisableTestParallelization = true)]
\ No newline at end of file
diff --git a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj b/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj
index 1ac6e73..8a15baf 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj
+++ b/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj
@@ -18,7 +18,7 @@ under the License.
   <PropertyGroup>
     <AssemblyName>Org.Apache.REEF.Bridge.JAR</AssemblyName>
     <Description>Helper project that builds the Bridge JAR </Description>
-    <TargetFramework>net461</TargetFramework>
+    <TargetFramework>netstandard2.0</TargetFramework>
   </PropertyGroup>
   <Import Project="..\build.props" />
   <PropertyGroup>
@@ -58,11 +58,15 @@ under the License.
       <!--The shaded jar of the bridge (client side) -->
       <Client_JAR_Name>reef-bridge-client-$(REEF_Version)-shaded.jar</Client_JAR_Name>
       <Client_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-client\target\$(Client_JAR_Name)</Client_JAR>
+	  <!--The shaded jar of the bridge core -->
+      <BridgeCore_JAR_Name>reef-bridge-proto-java-$(REEF_Version)-jar-with-dependencies.jar</BridgeCore_JAR_Name>
+      <BridgeCore_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-proto-java\target\$(BridgeCore_JAR_Name)</BridgeCore_JAR>
     </PropertyGroup>
-    <Exec Command="call &quot;$(M2_HOME)\bin\mvn.cmd&quot; --projects lang/java/reef-bridge-java,lang/java/reef-bridge-client --also-make -TC1 -DskipTests -P!code-quality install" Condition="!Exists('$(Bridge_JAR)')" WorkingDirectory="$(REEF_Source_Folder)" />
+    <Exec Command="call &quot;$(M2_HOME)\bin\mvn.cmd&quot; --projects lang/java/reef-bridge-java,lang/java/reef-bridge-client,lang/java/reef-bridge-proto-java --also-make -TC1 -DskipTests -P!code-quality install" Condition="!Exists('$(Bridge_JAR)') Or !Exists('$(BridgeCore_JAR)')" WorkingDirectory="$(REEF_Source_Folder)" />
     <Message Text="Copying jar files to $(OutputPath)" Importance="high" />
     <Copy DestinationFolder="$(OutputPath)" SourceFiles="$(Bridge_JAR)" />
     <Copy DestinationFolder="$(OutputPath)" SourceFiles="$(Client_JAR)" />
+	<Copy DestinationFolder="$(OutputPath)" SourceFiles="$(BridgeCore_JAR)" />
   </Target>
   <!--
     Executes 'mvn clean', if the JAR exists.
@@ -75,10 +79,14 @@ under the License.
       <!--The shaded jar of the bridge (client side) -->
       <Client_JAR_Name>reef-bridge-client-$(REEF_Version)-shaded.jar</Client_JAR_Name>
       <Client_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-client\target\$(Client_JAR_Name)</Client_JAR>
+	  <!--The shaded jar of the bridge core -->
+      <BridgeCore_JAR_Name>reef-bridge-proto-java-$(REEF_Version)-jar-with-dependencies.jar</BridgeCore_JAR_Name>
+      <BridgeCore_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-proto-java\target\$(Client_JAR_Name)</BridgeCore_JAR>
     </PropertyGroup>
-    <Exec Command="call &quot;$(M2_HOME)\bin\mvn.cmd&quot; -TC1 -DskipTests -q clean" Condition="Exists('$(Bridge_JAR)')" WorkingDirectory="$(REEF_Source_Folder)" />
+    <Exec Command="call &quot;$(M2_HOME)\bin\mvn.cmd&quot; -TC1 -DskipTests -q clean" Condition="Exists('$(Bridge_JAR)') Or Exists('$(BridgeCore_JAR)')" WorkingDirectory="$(REEF_Source_Folder)" />
     <Delete Files="$(OutputPath)\$(Bridge_JAR_Name)" />
     <Delete Files="$(OutputPath)\$(Client_JAR_Name)" />
+	    <Delete Files="$(OutputPath)\$(BridgeCore_JAR_Name)" />
   </Target>
   <Target Name="Rebuild" DependsOnTargets="Clean;Build" />
   <Target Name="CheckPrerequisites" DependsOnTargets="Build" />
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobParameters.cs b/lang/cs/Org.Apache.REEF.Client/API/JobParameters.cs
index 5279e3a..215dfeb 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/JobParameters.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobParameters.cs
@@ -32,6 +32,7 @@ namespace Org.Apache.REEF.Client.API
         private readonly string _jobIdentifier;
         private readonly int _maxApplicationSubmissions;
         private readonly int _driverMemory;
+        private readonly int _driverCpuCores;
         private IDictionary<string, string> _jobSubmissionEnvMap;
         private readonly Optional<string> _stdoutFilePath;
         private readonly Optional<string> _stderrFilePath;
@@ -41,6 +42,7 @@ namespace Org.Apache.REEF.Client.API
             string jobIdentifier, 
             int maxApplicationSubmissions, 
             int driverMemory,
+            int driverCpuCores,
             IDictionary<string, string> jobSubmissionEnvMap, 
             string stdoutFilePath,
             string stderrFilePath,
@@ -49,6 +51,7 @@ namespace Org.Apache.REEF.Client.API
             _jobIdentifier = jobIdentifier;
             _maxApplicationSubmissions = maxApplicationSubmissions;
             _driverMemory = driverMemory;
+            _driverCpuCores = driverCpuCores;
             _jobSubmissionEnvMap = jobSubmissionEnvMap;
 
             _stdoutFilePath = string.IsNullOrWhiteSpace(stdoutFilePath) ? 
@@ -86,6 +89,14 @@ namespace Org.Apache.REEF.Client.API
         }
 
         /// <summary>
+        /// The number of driver CPU cores.
+        /// </summary>
+        public int DriverCpuCores
+        {
+            get { return _driverCpuCores;  }
+        }
+
+        /// <summary>
         /// The job submission environment variable map.
         /// </summary>
         public IDictionary<string, string> JobSubmissionEnvMap
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobParametersBuilder.cs b/lang/cs/Org.Apache.REEF.Client/API/JobParametersBuilder.cs
index 39df14e..6b495fc 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/JobParametersBuilder.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobParametersBuilder.cs
@@ -27,6 +27,7 @@ namespace Org.Apache.REEF.Client.API
     {
         private string _jobIdentifier;
         private int _maxApplicationSubmissions = 1;
+        private int _driverCpuCores = 1;
         private int _driverMemory = 512;
         private string _stdoutFilePath = null;
         private string _stderrFilePath = null;
@@ -55,6 +56,7 @@ namespace Org.Apache.REEF.Client.API
                 _jobIdentifier, 
                 _maxApplicationSubmissions, 
                 _driverMemory,
+                _driverCpuCores,
                 _jobSubmissionMap,
                 _stdoutFilePath,
                 _stderrFilePath,
@@ -92,6 +94,17 @@ namespace Org.Apache.REEF.Client.API
         }
 
         /// <summary>
+        /// The number of cores to allocated for the driver
+        /// </summary>
+        /// <param name="driverCpuCores">number of cores</param>
+        /// <returns></returns>
+        public JobParametersBuilder SetDriverCpuCores(int driverCpuCores)
+        {
+            _driverCpuCores = driverCpuCores;
+            return this;
+        }
+
+        /// <summary>
         /// Set job submission environment variable.
         /// If the variable is already in the map, override it. 
         /// </summary>
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs b/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
index bc90c44..a571f2a 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
@@ -154,8 +154,15 @@ namespace Org.Apache.REEF.Client.Common
             var javaPath = Path.Combine(javaBinPath, "java.exe");
             if (!File.Exists(javaPath))
             {
-                throw new JavaNotFoundException(
-                    "Could not find java.exe on this machine. Is Java installed and JAVA_HOME set?", javaPath);
+                var linuxJavaPath = Path.Combine(javaBinPath, "java");
+                if (!File.Exists(linuxJavaPath))
+                {
+                    throw new JavaNotFoundException(
+                        "Could not find java executable on this machine. Is Java installed and JAVA_HOME set?",
+                        javaPath);
+                }
+
+                return linuxJavaPath;
             }
             return javaPath;
         }
diff --git a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
index b94fccb..5ad2c2d 100644
--- a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
+++ b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
@@ -93,7 +93,7 @@ under the License.
       var srcDir = @"lang\cs\Org.Apache.REEF.Client\Properties";
       var binDir = @"lang\cs\bin\.netcore";
       var resxInputPath  = Path.Combine(ProjectFolder, srcDir, "Resources.xml");      
-      var resourceDir = Path.Combine(ProjectFolder, binDir, DebugOrRelease, "Org.Apache.REEF.Bridge.JAR", "net461");
+      var resourceDir = Path.Combine(ProjectFolder, binDir, DebugOrRelease, "Org.Apache.REEF.Bridge.JAR", "netstandard2.0");
       var clrDriverDir = Path.Combine(ProjectFolder, binDir, DebugOrRelease, "Org.Apache.REEF.Bridge");
       var byteArrayType = ";System.Byte[], mscorlib, Version=4.0.0.0, Culture=neutral, PublicKeyToken=b77a5c561934e089";
       var jarRest = reefVersion + "-shaded.jar" + byteArrayType;
@@ -154,7 +154,7 @@ under the License.
     <Copy SourceFiles="@(MySourceFiles)" DestinationFiles="@(MySourceFiles->'$(TargetDir)%(Filename)%(Extension)')" />
   </Target>
   <Target Name="RemovingTempResx" AfterTargets="AfterBuild">
-    <Message Text="Removing temporary resx file" Importance="high"/>
+    <Message Text="Removing temporary resx file" Importance="high" />
     <Delete Files="$(TempResxFile)" />
   </Target>
 </Project>
diff --git a/lang/cs/Org.Apache.REEF.Client/Properties/AssemblyInfo.cs b/lang/cs/Org.Apache.REEF.Client/Properties/AssemblyInfo.cs
index 3c33105..3d70d6d 100644
--- a/lang/cs/Org.Apache.REEF.Client/Properties/AssemblyInfo.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Properties/AssemblyInfo.cs
@@ -21,6 +21,11 @@ using System.Runtime.CompilerServices;
 [assembly: InternalsVisibleTo("Org.Apache.REEF.Tests")]
 [assembly: InternalsVisibleTo("Org.Apache.REEF.Client.Tests")]
 
+// Allow bridge access to internal APIs
+[assembly: InternalsVisibleTo("Org.Apache.REEF.Bridge.Core")]
+[assembly: InternalsVisibleTo("Org.Apache.REEF.Bridge.Core.Client")]
+
+
 // Allow NSubstitute to create proxy implementations
 [assembly: InternalsVisibleTo("DynamicProxyGenAssembly2")]
 
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Common/Client/ICompletedJob.cs
similarity index 76%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Common/Client/ICompletedJob.cs
index 2f3352a..17f6926 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Client/ICompletedJob.cs
@@ -14,13 +14,14 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+using Org.Apache.REEF.Utilities;
 
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Common.Client
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <summary>
+    /// Event representing the completion of a job.
+    /// </summary>
+    public interface ICompletedJob : IIdentifiable
     {
     }
 }
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Common/Client/IFailedJob.cs
similarity index 76%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Common/Client/IFailedJob.cs
index 2f3352a..1dee514 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Client/IFailedJob.cs
@@ -14,13 +14,12 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+using Org.Apache.REEF.Common.Api;
+using Org.Apache.REEF.Utilities;
 
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Common.Client
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    public interface IFailedJob : IIdentifiable, IFailure
     {
     }
 }
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Common/Client/IFailedRuntime.cs
similarity index 76%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Common/Client/IFailedRuntime.cs
index 2f3352a..35a4dc8 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Client/IFailedRuntime.cs
@@ -14,13 +14,14 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+using Org.Apache.REEF.Common.Api;
 
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Common.Client
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    /// <summary>
+    /// Error message that REEF Client gets when there is an error in REEF resourcemanager.
+    /// </summary>
+    public interface IFailedRuntime : IFailure
     {
     }
 }
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Common/Client/IJobMessage.cs
similarity index 76%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Common/Client/IJobMessage.cs
index 2f3352a..f396323 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Client/IJobMessage.cs
@@ -14,13 +14,14 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+using System;
+using System.Collections.Generic;
+using System.Text;
+using Org.Apache.REEF.Utilities;
 
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Common.Client
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    public interface IJobMessage : IIdentifiable, IMessage
     {
     }
 }
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs b/lang/cs/Org.Apache.REEF.Common/Client/IRunningJob.cs
similarity index 60%
copy from lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
copy to lang/cs/Org.Apache.REEF.Common/Client/IRunningJob.cs
index 2f3352a..b0935b8 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Parameters/TcpPortRangeStart.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Client/IRunningJob.cs
@@ -14,13 +14,23 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+using System;
+using Org.Apache.REEF.Utilities;
 
-using Org.Apache.REEF.Tang.Annotations;
-
-namespace Org.Apache.REEF.Wake.Remote.Parameters
+namespace Org.Apache.REEF.Common.Client
 {
-    [NamedParameter(Documentation = "Port number range start for listening on tcp ports", DefaultValue = "8900")]
-    public sealed class TcpPortRangeStart : Name<int>
+    public interface IRunningJob : IIdentifiable, IDisposable
     {
+        /// <summary>
+        /// Cancels the running Job.
+        /// </summary>
+        /// <param name="message">delivered along with cancel request</param>
+        void Dispose(byte[] message);
+
+        /// <summary>
+        /// Send message to the driver.
+        /// </summary>
+        /// <param name="message">to deliver to the driver</param>
+        void Send(byte[] message);
     }
 }
diff --git a/lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs b/lang/cs/Org.Apache.REEF.Common/Client/ISubmittedJob.cs
similarity index 78%
copy from lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs
copy to lang/cs/Org.Apache.REEF.Common/Client/ISubmittedJob.cs
index e7b4768..0295960 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Properties/AssemblyInfo.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Client/ISubmittedJob.cs
@@ -1,4 +1,4 @@
-// Licensed to the Apache Software Foundation (ASF) under one
+// Licensed to the Apache Software Foundation (ASF) under one
 // or more contributor license agreements.  See the NOTICE file
... 4254 lines suppressed ...