You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ju...@apache.org on 2015/06/15 22:50:16 UTC

incubator-reef git commit: [REEF-220] Refactor DriverBridgeConfiguration

Repository: incubator-reef
Updated Branches:
  refs/heads/master 1aaf62d15 -> 281326ac2


[REEF-220] Refactor DriverBridgeConfiguration

This change addresses several items listed on [REEF-220]:

[REEF-356]:
  * Added `DriverConfiguration`
  * Removed deprecated event hanlders from that
  * Fixed the docs on HttpHandler [REEF-355]
  * Deprecated `DriverBridgeConfiguration`
  * Used `DriverConfiguration` in `HelloREEF`

[REEF-354]:
  * Introduced `IDriverStarted` and `IDriverRestarted`events to replace
    `DateTime` and `StartTime`

JIRAs:
  [REEF-220](https://issues.apache.org/jira/browse/REEF-220)
  [REEF-354](https://issues.apache.org/jira/browse/REEF-354)
  [REEF-355](https://issues.apache.org/jira/browse/REEF-355)
  [REEF-356](https://issues.apache.org/jira/browse/REEF-356)

Pull Request:
  This closes #210

Author:    Markus Weimer <we...@apache.org>


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

Branch: refs/heads/master
Commit: 281326ac2d9aae30e62c6e5e1df4d79ebc2e6a42
Parents: 1aaf62d
Author: Markus Weimer <we...@apache.org>
Authored: Wed Jun 10 10:41:13 2015 +1000
Committer: Julia Wang <jw...@yahoo.com>
Committed: Mon Jun 15 13:08:49 2015 -0700

----------------------------------------------------------------------
 .../Bridge/DriverBridge.cs                      |  21 +-
 .../Bridge/DriverBridgeConfiguration.cs         |   8 +-
 .../Bridge/DriverBridgeConfigurationOptions.cs  |  24 +--
 .../Bridge/DriverRestartHandlerWrapper.cs       |  60 ++++++
 .../Bridge/Events/DriverRestarted.cs            |  37 ++++
 .../Bridge/Events/DriverStarted.cs              |  41 ++++
 .../Defaults/DefaultDriverRestartedHandler.cs   |  48 +++++
 .../Defaults/DefaultDriverStartHandler.cs       |  54 -----
 .../Defaults/DefaultDriverStartedHandler.cs     |  52 +++++
 .../DriverConfiguration.cs                      | 210 +++++++++++++++++++
 .../Org.Apache.REEF.Driver/IDriverRestarted.cs  |  28 +++
 .../cs/Org.Apache.REEF.Driver/IDriverStarted.cs |  31 +++
 .../Org.Apache.REEF.Driver.csproj               |   9 +-
 .../HelloDriver.cs                              |   8 +-
 .../HelloREEF.cs                                |  11 +-
 15 files changed, 547 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
index dd6b443..2596c42 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
@@ -31,6 +31,7 @@ using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Utilities.Logging;
 using Org.Apache.REEF.Wake.Time.Event;
 using Org.Apache.REEF.Common.Evaluator.Parameters;
+using Org.Apache.REEF.Driver.Bridge.Events;
 
 namespace Org.Apache.REEF.Driver.Bridge
 {
@@ -70,9 +71,9 @@ namespace Org.Apache.REEF.Driver.Bridge
 
         private static ClrSystemHandler<StartTime> _driverRestartSubscriber;
 
-        private readonly ISet<IObserver<DateTime>> _driverStartHandlers;
+        private readonly ISet<IObserver<IDriverStarted>> _driverStartHandlers;
 
-        private readonly IObserver<StartTime> _driverRestartHandler; 
+        private readonly IObserver<StartTime> _legacyDriverRestartHandler;
 
         private readonly ISet<IObserver<IEvaluatorRequestor>> _evaluatorRequestHandlers;
 
@@ -110,8 +111,9 @@ namespace Org.Apache.REEF.Driver.Bridge
 
         [Inject]
         public DriverBridge(
-            [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverStartHandlers))] ISet<IObserver<DateTime>> driverStartHandlers,
-            [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverRestartHandler))] IObserver<StartTime> driverRestartHandler,
+            [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverStartedHandlers))] ISet<IObserver<IDriverStarted>> driverStartHandlers,
+            [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverRestartHandler))] IObserver<StartTime> legacyDriverRestartHandler,
+            [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverRestartedHandler))] IObserver<IDriverRestarted> driverRestartedHandler,
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.EvaluatorRequestHandlers))] ISet<IObserver<IEvaluatorRequestor>> evaluatorRequestHandlers,
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.AllocatedEvaluatorHandlers))] ISet<IObserver<IAllocatedEvaluator>> allocatedEvaluatorHandlers,
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.ActiveContextHandlers))] ISet<IObserver<IActiveContext>> activeContextHandlers,
@@ -163,7 +165,7 @@ namespace Org.Apache.REEF.Driver.Bridge
             _closedContextHandlers = closedContextHandlers;
             _failedContextHandlers = failedContextHandlers;
             _contextMessageHandlers = contextMessageHandlers;
-            _driverRestartHandler = driverRestartHandler;
+            _legacyDriverRestartHandler = new DriverRestartHandlerWrapper(legacyDriverRestartHandler, driverRestartedHandler);
             _driverRestartActiveContextHandlers = driverRestartActiveContextHandlers;
             _driverRestartRunningTaskHandlers = driverRestartRunningTaskHandlers;
             _httpServerHandler = httpServerHandler;
@@ -191,9 +193,9 @@ namespace Org.Apache.REEF.Driver.Bridge
         {
             ulong[] handlers = Enumerable.Repeat(Constants.NullHandler, Constants.HandlersNumber).ToArray();
 
-            // subscribe to StartTime event for driver restart         
-            _driverRestartSubscriber.Subscribe(_driverRestartHandler);
-            _logger.Log(Level.Info, "subscribed to Driver restart handler: " + _driverRestartHandler);
+            // subscribe to StartTime event for driver restart
+            _driverRestartSubscriber.Subscribe(_legacyDriverRestartHandler);
+            _logger.Log(Level.Info, "subscribed to Driver restart handler: " + _legacyDriverRestartHandler);
             handlers[Constants.Handlers[Constants.DriverRestartHandler]] = ClrHandlerHelper.CreateHandler(_driverRestartSubscriber);
 
             // subscribe to Allocated Evaluator
@@ -331,9 +333,10 @@ namespace Org.Apache.REEF.Driver.Bridge
         /// </summary>
         internal void StartHandlersOnNext(DateTime startTime)
         {
+            var driverStarted = new DriverStarted(startTime);
             foreach (var handler in _driverStartHandlers)
             {
-                handler.OnNext(startTime);
+                handler.OnNext(driverStarted);
                 _logger.Log(Level.Info, "called OnDriverStart handler: " + handler);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfiguration.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfiguration.cs
index f1029d4..78f0237 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfiguration.cs
@@ -37,6 +37,7 @@ using Org.Apache.REEF.Wake.Time.Event;
 
 namespace Org.Apache.REEF.Driver.Bridge
 {
+    [Obsolete(message:"Obsolete since 0.12, will be removed in 0,.13. Use DriverConfiguration instead.", error:false)]
     public class DriverBridgeConfiguration : ConfigurationModuleBuilder
     {
         /// <summary>
@@ -47,12 +48,6 @@ namespace Org.Apache.REEF.Driver.Bridge
         public static readonly OptionalImpl<IStartHandler> OnDriverStarted = new OptionalImpl<IStartHandler>();
 
         /// <summary>
-        /// The event handler called on driver start
-        /// </summary>
-        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
-        public static readonly OptionalImpl<IObserver<DateTime>> OnDriverStart = new OptionalImpl<IObserver<DateTime>>();
-
-        /// <summary>
         ///  The event handler invoked when driver restarts
         /// </summary>
         [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
@@ -213,7 +208,6 @@ namespace Org.Apache.REEF.Driver.Bridge
                 .BindImplementation(GenericType<IStartHandler>.Class, OnDriverStarted)
                 .BindNamedParameter(GenericType<DriverBridgeConfigurationOptions.DriverRestartHandler>.Class, OnDriverRestarted)
                 .BindImplementation(GenericType<IDriverConnection>.Class, OnDriverReconnect)
-                .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.DriverStartHandlers>.Class, OnDriverStart)
                 .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.EvaluatorRequestHandlers>.Class, OnEvaluatorRequested)
                 .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.AllocatedEvaluatorHandlers>.Class, OnEvaluatorAllocated)
                 .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.ActiveContextHandlers>.Class, OnContextActive)

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfigurationOptions.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfigurationOptions.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfigurationOptions.cs
index bbf7172..e10f49a 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfigurationOptions.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridgeConfigurationOptions.cs
@@ -36,22 +36,28 @@ namespace Org.Apache.REEF.Driver.Bridge
     /// <summary>
     /// Hosts all named parameters for Drivers, including bridge handlers.
     /// </summary>
-    public class DriverBridgeConfigurationOptions
+    public sealed class DriverBridgeConfigurationOptions
     {
         // Level.Verbose (since enum is not suppoted for TANG, we use a string here)
         private const string _verboseLevel = "Verbose";
 
-        // There is not supposed to be a default for Start handler but we need to provide one because all the existing apps would break;
-        [NamedParameter(documentation: "Called when driver is started, after CLR bridge is set up.", defaultClasses: new[] { typeof(DefaultDriverStartHandler) })]
-        public class DriverStartHandlers : Name<ISet<IObserver<DateTime>>>
+        // TODO: Remove the default value in 0.13 when the DriverStartedHandler becomes mandatory
+        [NamedParameter(documentation:"The start point for application logic. Event fired after the Driver is done initializing.", defaultClasses: new []{typeof(DefaultDriverStartedHandler)})]
+        public class DriverStartedHandlers : Name<ISet<IObserver<IDriverStarted>>>
         {
         }
 
+        [Obsolete(message:"Since 0.12, Removed in 0.13. Use DriverRestartedHandler instead.")]
         [NamedParameter(documentation: "Called when driver is restarted, after CLR bridge is set up.", defaultClasses: new[] { typeof(DefaultDriverRestartHandler) })]
         public class DriverRestartHandler : Name<IObserver<StartTime>>
         {
         }
 
+        [NamedParameter(documentation: "Called when driver is restarted, after CLR bridge is set up.", defaultClasses: new[] { typeof(DefaultDriverRestartedHandler) })]
+        public class DriverRestartedHandler : Name<IObserver<IDriverRestarted>>
+        {
+        }
+
         [NamedParameter(documentation: "Called when evaluator is requested.")] 
         public class EvaluatorRequestHandlers : Name<ISet<IObserver<IEvaluatorRequestor>>>
         {
@@ -146,15 +152,5 @@ namespace Org.Apache.REEF.Driver.Bridge
         public class TraceLevel : Name<string>
         {
         }
-
-        //[NamedParameter(documentation: "Job message handler.", defaultClasses: new[] { typeof(DefaultClientMessageHandler) })]
-        //public class ClientMessageHandlers : Name<ISet<IObserver<byte[]>>>
-        //{
-        //}
-
-        //[NamedParameter(documentation: "Client close handler.", defaultClasses: new[] { typeof(DefaultClientCloseHandler) })]
-        //public class ClientCloseHandlers : Name<ISet<IObserver<byte[]>>>
-        //{
-        //}
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverRestartHandlerWrapper.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverRestartHandlerWrapper.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverRestartHandlerWrapper.cs
new file mode 100644
index 0000000..7ff1ea4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverRestartHandlerWrapper.cs
@@ -0,0 +1,60 @@
+/**
+ * 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 Org.Apache.REEF.Driver.Bridge.Events;
+using Org.Apache.REEF.Wake.Time.Event;
+
+namespace Org.Apache.REEF.Driver.Bridge
+{
+    /// <summary>
+    /// Wrapper of the new Observers of DriverRestarted with and into an Observer of StartTime.
+    /// </summary>
+    /// <remarks>
+    /// Rationale: This way, we don't have to change the C++ code in the same change as the API.
+    /// </remarks>
+    internal sealed class DriverRestartHandlerWrapper : IObserver<StartTime>
+    {
+        private readonly IObserver<IDriverRestarted> _driverRestartedObserver;
+        private readonly IObserver<StartTime> _startTimeObserver;
+
+        internal DriverRestartHandlerWrapper(IObserver<StartTime> startTimeObserver,
+            IObserver<IDriverRestarted> driverRestartedObserver)
+        {
+            _startTimeObserver = startTimeObserver;
+            _driverRestartedObserver = driverRestartedObserver;
+        }
+
+        public void OnNext(StartTime startTime)
+        {
+            _driverRestartedObserver.OnNext(new DriverRestarted(new DateTime(startTime.TimeStamp)));
+            _startTimeObserver.OnNext(startTime);
+        }
+
+        public void OnError(Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverRestarted.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverRestarted.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverRestarted.cs
new file mode 100644
index 0000000..5f8fdcc
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverRestarted.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;
+
+namespace Org.Apache.REEF.Driver.Bridge.Events
+{
+    internal sealed class DriverRestarted : IDriverRestarted
+    {
+        private readonly DateTime _startTime;
+
+        internal DriverRestarted(DateTime startTime)
+        {
+            _startTime = startTime;
+        }
+
+        public DateTime StartTime
+        {
+            get { return _startTime; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverStarted.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverStarted.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverStarted.cs
new file mode 100644
index 0000000..9cc41b4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/DriverStarted.cs
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+
+namespace Org.Apache.REEF.Driver.Bridge.Events
+{
+    /// <summary>
+    /// Implementation of IDriverStarted.
+    /// </summary>
+    internal sealed class DriverStarted : IDriverStarted
+    {
+        private readonly DateTime _startTime;
+
+        internal DriverStarted(DateTime startTime)
+        {
+            _startTime = startTime;
+        }
+
+        public DateTime StartTime
+        {
+            get { return _startTime; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverRestartedHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverRestartedHandler.cs b/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverRestartedHandler.cs
new file mode 100644
index 0000000..537ac9b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverRestartedHandler.cs
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Driver.Defaults
+{
+    internal sealed class DefaultDriverRestartedHandler : IObserver<IDriverRestarted>
+    {
+        [Inject]
+        private DefaultDriverRestartedHandler()
+        {
+        }
+
+        public void OnNext(IDriverRestarted value)
+        {
+            // We throw immediately, as there is no sane default for Driver restart behavior.
+            throw new Exception("Driver restart handler was called, but not implemented.");
+        }
+
+        public void OnError(Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartHandler.cs b/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartHandler.cs
deleted file mode 100644
index 8d94941..0000000
--- a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartHandler.cs
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-using System;
-using Org.Apache.REEF.Tang.Annotations;
-using Org.Apache.REEF.Utilities.Logging;
-using Org.Apache.REEF.Wake.Time.Event;
-
-namespace Org.Apache.REEF.Driver.Defaults
-{
-    /// <summary>
-    ///  Default event handler for driver start: Logging it.
-    /// </summary>
-    public class DefaultDriverStartHandler : IObserver<DateTime>
-    {
-        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultDriverStartHandler));
-
-        [Inject]
-        public DefaultDriverStartHandler()
-        {
-        }
-
-        public void OnNext(DateTime startTime)
-        {
-            LOGGER.Log(Level.Info, "Driver started at" + startTime);
-        }
-
-        public void OnError(Exception error)
-        {
-            throw new NotImplementedException();
-        }
-
-        public void OnCompleted()
-        {
-            throw new NotImplementedException();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartedHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartedHandler.cs b/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartedHandler.cs
new file mode 100644
index 0000000..b32106e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/Defaults/DefaultDriverStartedHandler.cs
@@ -0,0 +1,52 @@
+/**
+ * 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 Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Driver.Defaults
+{
+    // This class should be removed in 0.13, when the DriverStartedHandler becomes mandatory.
+    [Obsolete("This class will be removed in 0.13.", false)]
+    internal sealed class DefaultDriverStartedHandler : IObserver<IDriverStarted>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultDriverStartedHandler));
+
+        [Inject]
+        private DefaultDriverStartedHandler()
+        {
+        }
+
+        public void OnNext(IDriverStarted value)
+        {
+            LOGGER.Log(Level.Info, "Driver started at {0}", value.StartTime);
+        }
+
+        public void OnError(Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/DriverConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/DriverConfiguration.cs b/lang/cs/Org.Apache.REEF.Driver/DriverConfiguration.cs
new file mode 100644
index 0000000..206d0e9
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/DriverConfiguration.cs
@@ -0,0 +1,210 @@
+/**
+ * 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.Diagnostics;
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Common.Evaluator;
+using Org.Apache.REEF.Driver.Bridge;
+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.Wake.Time.Event;
+
+namespace Org.Apache.REEF.Driver
+{
+    /// <summary>
+    /// Fill this out to configure a Driver.
+    /// </summary>
+    public sealed class DriverConfiguration : 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 invoked when driver restarts
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<StartTime>> OnDriverRestarted =
+            new OptionalImpl<IObserver<StartTime>>();
+
+        /// <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 for HTTP calls to the Driver's HTTP server.
+        /// </summary>
+        public static readonly OptionalImpl<IHttpHandler> OnHttpEvent = new OptionalImpl<IHttpHandler>();
+
+        /// <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>> OnDirverRestartContextActive =
+            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>
+        /// Additional set of string arguments that can be pssed to handlers through client
+        /// </summary>
+        public static readonly OptionalParameter<string> CommandLineArguments = new OptionalParameter<string>();
+
+        /// <summary>
+        /// The trace level of the TraceListner
+        /// </summary>
+        public static readonly OptionalParameter<string> CustomTraceLevel = new OptionalParameter<string>();
+
+        /// <summary>
+        /// Additional set of trace listners provided by client
+        /// </summary>
+        public static readonly OptionalParameter<TraceListener> CustomTraceListeners =
+            new OptionalParameter<TraceListener>();
+
+        /// <summary>
+        /// The implemenation for (attempting to) re-establish connection to driver
+        /// </summary>
+        public static readonly OptionalImpl<IDriverConnection> OnDriverReconnect = new OptionalImpl<IDriverConnection>();
+
+        public static ConfigurationModule ConfigurationModule
+        {
+            get
+            {
+                return new DriverConfiguration()
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.DriverStartedHandlers>.Class,
+                        OnDriverStarted)
+                    .BindNamedParameter(GenericType<DriverBridgeConfigurationOptions.DriverRestartHandler>.Class,
+                        OnDriverRestarted)
+                    .BindImplementation(GenericType<IDriverConnection>.Class, OnDriverReconnect)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.AllocatedEvaluatorHandlers>.Class,
+                        OnEvaluatorAllocated)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.ActiveContextHandlers>.Class,
+                        OnContextActive)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.TaskMessageHandlers>.Class, OnTaskMessage)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.FailedTaskHandlers>.Class, OnTaskFailed)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.RunningTaskHandlers>.Class, OnTaskRunning)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.SuspendedTaskHandlers>.Class,
+                        OnTaskSuspended)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.FailedEvaluatorHandlers>.Class,
+                        OnEvaluatorFailed)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.CompletedEvaluatorHandlers>.Class,
+                        OnEvaluatorCompleted)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.CompletedTaskHandlers>.Class,
+                        OnTaskCompleted)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.ClosedContextHandlers>.Class,
+                        OnContextClosed)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.FailedContextHandlers>.Class,
+                        OnContextFailed)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.ContextMessageHandlers>.Class,
+                        OnContextMessage)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.ArgumentSets>.Class, CommandLineArguments)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.HttpEventHandlers>.Class, OnHttpEvent)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.TraceListenersSet>.Class,
+                        CustomTraceListeners)
+                    .BindSetEntry(
+                        GenericType<DriverBridgeConfigurationOptions.DriverRestartActiveContextHandlers>.Class,
+                        OnDirverRestartContextActive)
+                    .BindSetEntry(GenericType<DriverBridgeConfigurationOptions.DriverRestartRunningTaskHandlers>.Class,
+                        OnDriverRestartTaskRunning)
+                    .BindNamedParameter(GenericType<DriverBridgeConfigurationOptions.TraceLevel>.Class, CustomTraceLevel)
+                    .Build();
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/IDriverRestarted.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/IDriverRestarted.cs b/lang/cs/Org.Apache.REEF.Driver/IDriverRestarted.cs
new file mode 100644
index 0000000..63bf61a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/IDriverRestarted.cs
@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+namespace Org.Apache.REEF.Driver
+{
+    /// <summary>
+    /// Event fired on Driver restarts instead of IDriverStarted.
+    /// </summary>
+    public interface IDriverRestarted : IDriverStarted
+    {
+         
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/IDriverStarted.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/IDriverStarted.cs b/lang/cs/Org.Apache.REEF.Driver/IDriverStarted.cs
new file mode 100644
index 0000000..5010f0f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/IDriverStarted.cs
@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+namespace Org.Apache.REEF.Driver
+{
+    /// <summary>
+    /// Event fired when the Driver started.
+    /// </summary>
+    public interface IDriverStarted
+    {
+        DateTime StartTime { get; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj b/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
index b334930..4c6fdb6 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
+++ b/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
@@ -62,12 +62,15 @@ under the License.
     <Compile Include="Bridge\DriverBridge.cs" />
     <Compile Include="Bridge\DriverBridgeConfiguration.cs" />
     <Compile Include="Bridge\DriverBridgeConfigurationOptions.cs" />
+    <Compile Include="Bridge\DriverRestartHandlerWrapper.cs" />
     <Compile Include="Bridge\Events\ActiveContext.cs" />
     <Compile Include="Bridge\Events\AllocatedEvaluator.cs" />
     <Compile Include="Bridge\Events\ClosedContext.cs" />
     <Compile Include="Bridge\Events\CompletedEvaluator.cs" />
     <Compile Include="Bridge\Events\CompletedTask.cs" />
     <Compile Include="Bridge\Events\ContextMessage.cs" />
+    <Compile Include="Bridge\Events\DriverRestarted.cs" />
+    <Compile Include="Bridge\Events\DriverStarted.cs" />
     <Compile Include="Bridge\Events\EvaluatorRequstor.cs" />
     <Compile Include="Bridge\Events\FailedContext.cs" />
     <Compile Include="Bridge\Events\FailedEvaluator.cs" />
@@ -104,9 +107,10 @@ under the License.
     <Compile Include="Defaults\DefaultContextMessageHandler.cs" />
     <Compile Include="Defaults\DefaultCustomTraceListener.cs" />
     <Compile Include="Defaults\DefaultDriverRestartContextActiveHandler.cs" />
+    <Compile Include="Defaults\DefaultDriverRestartedHandler.cs" />
     <Compile Include="Defaults\DefaultDriverRestartHandler.cs" />
     <Compile Include="Defaults\DefaultDriverRestartTaskRunningHandler.cs" />
-    <Compile Include="Defaults\DefaultDriverStartHandler.cs" />
+    <Compile Include="Defaults\DefaultDriverStartedHandler.cs" />
     <Compile Include="Defaults\DefaultEvaluatorAllocationHandler.cs" />
     <Compile Include="Defaults\DefaultEvaluatorCompletionHandler.cs" />
     <Compile Include="Defaults\DefaultEvaluatorFailureHandler.cs" />
@@ -119,6 +123,7 @@ under the License.
     <Compile Include="Defaults\DefaultTaskRunningHandler.cs" />
     <Compile Include="Defaults\DefaultTaskSuspensionHandler.cs" />
     <Compile Include="DriverConfigGenerator.cs" />
+    <Compile Include="DriverConfiguration.cs" />
     <Compile Include="DriverConfigurationSettings.cs" />
     <Compile Include="DriverSubmissionSettings.cs" />
     <Compile Include="Evaluator\EvaluatorDescriptorImpl.cs" />
@@ -132,6 +137,8 @@ under the License.
     <Compile Include="Evaluator\IFailedEvaluator.cs" />
     <Compile Include="FailedJob.cs" />
     <Compile Include="IDriver.cs" />
+    <Compile Include="IDriverRestarted.cs" />
+    <Compile Include="IDriverStarted.cs" />
     <Compile Include="IStartHandler.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
     <Compile Include="Task\ICompletedTask.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
index 34a25a8..a5455be 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
@@ -36,7 +36,7 @@ namespace Org.Apache.REEF.Examples.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<DateTime>
+    public sealed class HelloDriver : IObserver<IAllocatedEvaluator>, IObserver<IDriverStarted>
     {
         private static readonly Logger _Logger = Logger.GetLogger(typeof(HelloDriver));
 
@@ -70,10 +70,10 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         /// <summary>
         /// Called to start the user mode driver
         /// </summary>
-        /// <param name="startTime"></param>
-        public void OnNext(DateTime startTime)
+        /// <param name="driverStarted"></param>
+        public void OnNext(IDriverStarted driverStarted)
         {
-            _Logger.Log(Level.Info, string.Format("HelloDriver started at {0}", startTime));
+            _Logger.Log(Level.Info, string.Format("HelloDriver started at {0}", driverStarted.StartTime));
             _evaluatorRequestor.Submit(new EvaluatorRequest(number: 1, megaBytes: 64));
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/281326ac/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
index dbc3379..e71356b 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
@@ -21,8 +21,7 @@ using System;
 using Org.Apache.REEF.Client.API;
 using Org.Apache.REEF.Client.Local;
 using Org.Apache.REEF.Client.YARN;
-using Org.Apache.REEF.Common.Io;
-using Org.Apache.REEF.Driver.Bridge;
+using Org.Apache.REEF.Driver;
 using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Tang.Interface;
@@ -53,14 +52,14 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         private void Run()
         {
             // The driver configuration contains all the needed bindings.
-            var helloDriverConfiguration = DriverBridgeConfiguration.ConfigurationModule
-                .Set(DriverBridgeConfiguration.OnEvaluatorAllocated, GenericType<HelloDriver>.Class)
-                .Set(DriverBridgeConfiguration.OnDriverStart, GenericType<HelloDriver>.Class)
+            var helloDriverConfiguration = DriverConfiguration.ConfigurationModule
+                .Set(DriverConfiguration.OnEvaluatorAllocated, GenericType<HelloDriver>.Class)
+                .Set(DriverConfiguration.OnDriverStarted, GenericType<HelloDriver>.Class)
                 .Build();
             // The JobSubmission contains the Driver configuration as well as the files needed on the Driver.
             var helloJobSubmission = _jobSubmissionBuilderFactory.GetJobSubmissionBuilder()
                 .AddDriverConfiguration(helloDriverConfiguration)
-                .AddGlobalAssemblyForType(typeof (HelloDriver))
+                .AddGlobalAssemblyForType(typeof(HelloDriver))
                 .SetJobIdentifier("HelloREEF")
                 .Build();