You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by we...@apache.org on 2015/01/29 21:43:06 UTC

[22/31] incubator-reef git commit: [REEF-97] Add the REEF.NET code base

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedTask.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedTask.cs
new file mode 100644
index 0000000..5b34349
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedTask.cs
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Task;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public class FailedTask : IFailedTask
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(FailedTask));
+        
+        public FailedTask(IFailedTaskClr2Java failedTaskClr2Java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            Parse(failedTaskClr2Java);
+            FailedTaskClr2Java = failedTaskClr2Java;
+            ActiveContextClr2Java = failedTaskClr2Java.GetActiveContext();
+        }
+
+        public Optional<string> Reason { get; set; }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        public string Id { get; set; }
+
+        public string Message { get; set; }
+
+        public Optional<string> Description { get; set; }
+
+        public Optional<Exception> Cause { get; set; }
+
+        public Optional<byte[]> Data { get; set; }
+
+        [DataMember]
+        private IFailedTaskClr2Java FailedTaskClr2Java { get; set; }
+
+        [DataMember]
+        private IActiveContextClr2Java ActiveContextClr2Java { get; set; }
+
+        /// <summary>
+        /// Access the context the task ran (and crashed) on, if it could be recovered.
+        /// An ActiveContext is given when the task fails but the context remains alive.
+        /// On context failure, the context also fails and is surfaced via the FailedContext event.
+        /// Note that receiving an ActiveContext here is no guarantee that the context (and evaluator)
+        /// are in a consistent state. Application developers need to investigate the reason available
+        /// via getCause() to make that call.
+        /// return the context the Task ran on.
+        /// </summary>
+        public Optional<IActiveContext> GetActiveContext()
+        {
+            IActiveContext activeContext = new ActiveContext(ActiveContextClr2Java);
+            return ActiveContextClr2Java == null ? Optional<IActiveContext>.Empty() : Optional<IActiveContext>.Of(activeContext);
+        }
+
+        public Exception AsError()
+        {
+            throw new NotImplementedException();
+        }
+
+        private void Parse(IFailedTaskClr2Java failedTaskClr2Java)
+        {
+            string serializedInfo = failedTaskClr2Java.GetString();
+            LOGGER.Log(Level.Verbose, "serialized failed task: " + serializedInfo);
+            Dictionary<string, string> settings = new Dictionary<string, string>();
+            string[] components = serializedInfo.Split(',');
+            foreach (string component in components)
+            {
+                string[] pair = component.Trim().Split('=');
+                if (pair == null || pair.Length != 2)
+                {
+                    Exceptions.Throw(new ArgumentException("invalid component to be used as key-value pair:", component), LOGGER);
+                }
+                settings.Add(pair[0], pair[1]);
+            }
+
+            string id;
+            if (!settings.TryGetValue("Identifier", out id))
+            {
+                Exceptions.Throw(new ArgumentException("cannot find Identifier entry."), LOGGER);
+            }
+            Id = id;
+
+            string msg;
+            if (!settings.TryGetValue("Message", out msg))
+            {
+                LOGGER.Log(Level.Verbose, "no Message in Failed Task.");
+                msg = string.Empty;
+            }
+            Message = msg;
+
+            string description;
+            if (!settings.TryGetValue("Description", out description))
+            {
+                LOGGER.Log(Level.Verbose, "no Description in Failed Task.");
+                description = string.Empty;
+            }
+            Description = string.IsNullOrWhiteSpace(description) ? Optional<string>.Empty() : Optional<string>.Of(description);
+
+            string cause;
+            if (!settings.TryGetValue("Cause", out cause))
+            {
+                LOGGER.Log(Level.Verbose, "no Cause in Failed Task.");
+                cause = string.Empty;
+            }
+            Reason = string.IsNullOrWhiteSpace(cause) ? Optional<string>.Empty() : Optional<string>.Of(cause);
+
+            string rawData;
+            if (!settings.TryGetValue("Data", out rawData))
+            {
+                LOGGER.Log(Level.Verbose, "no Data in Failed Task.");
+                rawData = string.Empty;
+            }
+            Data = string.IsNullOrWhiteSpace(rawData) ? Optional<byte[]>.Empty() : Optional<byte[]>.Of(ByteUtilities.StringToByteArrays(rawData));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/RunningTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/RunningTask.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/RunningTask.cs
new file mode 100644
index 0000000..a9efa29
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/RunningTask.cs
@@ -0,0 +1,97 @@
+/**
+ * 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.Task;
+using Org.Apache.Reef.Utilities.Logging;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public class RunningTask : IRunningTask
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(RunningTask));
+        private IRunningTaskClr2Java _runningTaskClr2Java;
+        private IActiveContextClr2Java _activeContextClr2Java;
+
+        public RunningTask(IRunningTaskClr2Java runningTaskClr2Java)
+        {
+            using (LOGGER.LogFunction("RunningTask::RunningTask"))
+            {
+                _runningTaskClr2Java = runningTaskClr2Java;
+                _activeContextClr2Java = runningTaskClr2Java.GetActiveContext();
+            }
+        }
+
+        public Context.IActiveContext ActiveContext
+        {
+            get
+            {
+                return new ActiveContext(_activeContextClr2Java);
+            }
+            
+            set
+            {
+                ActiveContext = value;
+            }
+        }
+
+        public string Id
+        {
+            get
+            {
+                return _runningTaskClr2Java.GetId();
+            }
+
+            set
+            {
+                Id = value;
+            }
+        }
+
+        public void Send(byte[] message)
+        {
+            _runningTaskClr2Java.Send(message);
+        }
+
+        public void OnNext(byte[] message)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void Suspend(byte[] message)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void Suspend()
+        {
+            throw new NotImplementedException();
+        }
+
+        public void Dispose(byte[] message)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void Dispose()
+        {
+            throw new NotImplementedException();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/SuspendedTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/SuspendedTask.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/SuspendedTask.cs
new file mode 100644
index 0000000..33f7b8f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/SuspendedTask.cs
@@ -0,0 +1,81 @@
+/**
+ * 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 System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    [DataContract]
+    internal class SuspendedTask : ISuspendedTask
+    {
+        internal SuspendedTask(ISuspendedTaskClr2Java suspendedTaskClr2Java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            SuspendedTaskClr2Java = suspendedTaskClr2Java;
+            ActiveContextClr2Java = suspendedTaskClr2Java.GetActiveContext();
+        }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        public byte[] Message
+        {
+            get
+            {
+                return SuspendedTaskClr2Java.Get();
+            }
+
+            set
+            {
+            }
+        }
+
+        public string Id
+        {
+            get
+            {
+                return SuspendedTaskClr2Java.GetId();
+            }
+
+            set
+            {
+            }
+        }
+
+        public IActiveContext ActiveContext
+        {
+            get
+            {
+                return new ActiveContext(ActiveContextClr2Java);
+            }
+
+            set
+            {
+            }
+        }
+
+        [DataMember]
+        private ISuspendedTaskClr2Java SuspendedTaskClr2Java { get; set; }
+
+        [DataMember]
+        private IActiveContextClr2Java ActiveContextClr2Java { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/TaskMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/TaskMessage.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/TaskMessage.cs
new file mode 100644
index 0000000..f12039e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/TaskMessage.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.Task;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    /// <summary>
+    /// TaskMessage which wraps ITaskMessageClr2Java
+    /// </summary>
+    [DataContract]
+    internal class TaskMessage : ITaskMessage
+    {
+        private ITaskMessageClr2Java _taskMessageClr2Java;
+        private byte[] _message;
+        private string _instanceId;
+
+        public TaskMessage(ITaskMessageClr2Java clr2Java, byte[] message)
+        {
+            _instanceId = Guid.NewGuid().ToString("N");
+            _taskMessageClr2Java = clr2Java;
+            _message = message;
+        }
+
+        [DataMember]
+        public string InstanceId
+        {
+            get { return _instanceId; }
+            set { _instanceId = value; }
+        }
+
+        [DataMember]
+        public string TaskId
+        {
+            get { return _taskMessageClr2Java.GetId(); }
+            set { }
+        }
+
+        [DataMember]
+        public byte[] Message
+        {
+            get { return _message; }
+            set { _message = value; } 
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfiguration.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfiguration.cs
new file mode 100644
index 0000000..451ad2a
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfiguration.cs
@@ -0,0 +1,93 @@
+/**
+ * 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.Events;
+using Org.Apache.Reef.Tasks;
+using Org.Apache.Reef.Tasks.Events;
+using Org.Apache.Reef.Tang.Formats;
+using Org.Apache.Reef.Tang.Util;
+using System;
+using System.Diagnostics.CodeAnalysis;
+
+[module: SuppressMessage("StyleCop.CSharp.MaintainabilityRules", "SA1401:FieldsMustBePrivate", Justification = "static field, typical usage in configurations")]
+
+namespace Org.Apache.Reef.Driver.Context
+{
+    public class ContextConfiguration : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        ///  The identifier of the context.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly RequiredParameter<string> Identifier = new RequiredParameter<string>();
+
+        /// <summary>
+        ///  for context start. Defaults to logging if not bound.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly OptionalImpl<IObserver<IContextStart>> OnContextStart = new OptionalImpl<IObserver<IContextStart>>();
+
+        /// <summary>
+        /// for context stop. Defaults to logging if not bound.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly OptionalImpl<IObserver<IContextStop>> OnContextStop = new OptionalImpl<IObserver<IContextStop>>();
+
+        /// <summary>
+        ///  to be informed right before a Task enters its call() method.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly OptionalImpl<IObserver<ITaskStart>> OnTaskStart = new OptionalImpl<IObserver<ITaskStart>>();
+
+        /// <summary>
+        ///  to be informed right after a Task exits its call() method.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly OptionalImpl<IObserver<ITaskStop>> OnTaskStop = new OptionalImpl<IObserver<ITaskStop>>();
+
+        /// <summary>
+        ///  Source of messages to be called whenever the evaluator is about to make a heartbeat.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly OptionalImpl<IContextMessageSource> OnSendMessage = new OptionalImpl<IContextMessageSource>();
+
+        /// <summary>
+        ///   Driver has sent the context a message, and this parameter is used to register a handler on the context for processing that message.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly OptionalImpl<IContextMessageHandler> OnMessage = new OptionalImpl<IContextMessageHandler>();
+
+        public static ConfigurationModule ConfigurationModule
+        {
+            get
+            {
+                return new ContextConfiguration()
+                    .BindNamedParameter(GenericType<ContextConfigurationOptions.ContextIdentifier>.Class, Identifier)
+                    .BindSetEntry(GenericType<ContextConfigurationOptions.StartHandlers>.Class, OnContextStart)
+                    .BindSetEntry(GenericType<ContextConfigurationOptions.StopHandlers>.Class, OnContextStop)
+                    .BindSetEntry(GenericType<ContextConfigurationOptions.ContextMessageSources>.Class, OnSendMessage)
+                    .BindSetEntry(GenericType<ContextConfigurationOptions.ContextMessageHandlers>.Class, OnMessage)
+                    .BindSetEntry(GenericType<TaskConfigurationOptions.StartHandlers>.Class, OnTaskStart)
+                    .BindSetEntry(GenericType<TaskConfigurationOptions.StopHandlers>.Class, OnTaskStop)
+                    .Build();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfigurationOptions.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfigurationOptions.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfigurationOptions.cs
new file mode 100644
index 0000000..66d7de7
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/ContextConfigurationOptions.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 Org.Apache.Reef.Common.Context;
+using Org.Apache.Reef.Common.Events;
+using Org.Apache.Reef.Driver.Defaults;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Tang.Formats;
+using System;
+using System.Collections.Generic;
+
+namespace Org.Apache.Reef.Driver.Context
+{
+    /// <summary>
+    ///  Configuration parameters for ContextConfiguration module.
+    /// </summary>
+    public class ContextConfigurationOptions : ConfigurationModuleBuilder 
+    {
+        [NamedParameter(documentation: "The identifier for the context.")]
+        public class ContextIdentifier : Name<string>
+        {
+        }
+
+        [NamedParameter(documentation: "The set of event handlers for the ContextStart event", defaultClasses: new[] { typeof(DefaultContextStartHandler) })]
+        public class StartHandlers : Name<ISet<IObserver<IContextStart>>>
+        {
+        }
+
+        [NamedParameter(documentation: "The set of event handlers for the ContextStop event", defaultClasses: new[] { typeof(DefaultContextStopHandler) })]
+        public class StopHandlers : Name<ISet<IObserver<IContextStop>>>
+        {
+        }
+
+        [NamedParameter(documentation: "The set of ContextMessageSource implementations called during heartbeats.", defaultClasses: new[] { typeof(DefaultContextMessageSource) })]
+        public class ContextMessageSources : Name<ISet<IContextMessageSource>>
+        {
+        }
+
+        [NamedParameter(documentation: "The set of Context message handlers.")]
+        public class ContextMessageHandlers : Name<ISet<IContextMessageHandler>>
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/EvaluatorContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/EvaluatorContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/EvaluatorContext.cs
new file mode 100644
index 0000000..1bd8b18
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/EvaluatorContext.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.Common.ProtoBuf.EvaluatorRunTimeProto;
+using Org.Apache.Reef.Driver.Bridge;
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Interface;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Driver
+{
+    public class EvaluatorContext : IActiveContext
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(EvaluatorContext));
+        
+        private string _identifier;
+
+        private Optional<string> _parentId;
+
+        private EvaluatorManager _evaluatorManager;
+
+        private bool _disposed = false;
+
+        public EvaluatorContext(EvaluatorManager evaluatorManager, string id, Optional<string> parentId)
+        {
+            _identifier = id;
+            _parentId = parentId;
+            _evaluatorManager = evaluatorManager;
+        }
+
+        public string Id
+        {
+            get
+            {
+                return _identifier;
+            }
+
+            set
+            {
+            }
+        }
+
+        public string EvaluatorId
+        {
+            get
+            {
+                return _evaluatorManager.Id;
+            }
+
+            set
+            {
+            }
+        }
+
+        public Optional<string> ParentId
+        {
+            get
+            {
+                return _parentId;
+            }
+
+            set
+            {
+            }
+        }
+
+        public IEvaluatorDescriptor EvaluatorDescriptor
+        {
+            get
+            {
+                return _evaluatorManager.EvaluatorDescriptor;
+            }
+
+            set
+            {
+            }
+        }
+
+        public void Dispose()
+        {
+            if (_disposed)
+            {
+                var e = new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Active context [{0}] already closed", _identifier));
+                Exceptions.Throw(e, LOGGER);
+            }
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "Submit close context: RunningEvaluator id [{0}] for context id [{1}]", EvaluatorId, Id));
+            RemoveContextProto removeContextProto = new RemoveContextProto();
+            removeContextProto.context_id = Id;
+            ContextControlProto contextControlProto = new ContextControlProto();
+            contextControlProto.remove_context = removeContextProto;
+            _evaluatorManager.Handle(contextControlProto);
+            _disposed = true;
+        }
+
+        public ClosedContext GetClosedContext(IActiveContext parentContext)
+        {
+            //return new ClosedContext(parentContext, EvaluatorId, Id, ParentId, EvaluatorDescriptor);
+            throw new NotImplementedException();
+        }
+
+        public FailedContext GetFailedContext(Optional<IActiveContext> parentContext, Exception cause)
+        {
+            //return new FailedContext(parentContext, Id, cause, EvaluatorId, ParentId, EvaluatorDescriptor);
+            throw new NotImplementedException();
+        }
+
+        public void SubmitTask(IConfiguration taskConf)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void SubmitContext(IConfiguration contextConfiguration)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void SubmitContextAndService(IConfiguration contextConfiguration, IConfiguration serviceConfiguration)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void SendMessage(byte[] message)
+        {
+            throw new NotImplementedException();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/IActiveContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/IActiveContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IActiveContext.cs
new file mode 100644
index 0000000..b511e25
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IActiveContext.cs
@@ -0,0 +1,29 @@
+/**
+ * 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;
+using System;
+
+namespace Org.Apache.Reef.Driver.Context
+{
+    public interface IActiveContext : IDisposable, IContext, ITaskSubmittable, IContextSubmittable
+    {
+        void SendMessage(byte[] message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/IClosedContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/IClosedContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IClosedContext.cs
new file mode 100644
index 0000000..8ea7cc2
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IClosedContext.cs
@@ -0,0 +1,26 @@
+/**
+ * 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.Context
+{
+    public interface IClosedContext : IContext
+    {
+        IActiveContext ParentContext { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/IContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/IContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IContext.cs
new file mode 100644
index 0000000..d47b9ea
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IContext.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.Evaluator;
+using Org.Apache.Reef.Utilities;
+
+namespace Org.Apache.Reef.Driver.Context
+{
+    /// <summary>
+    /// A common base interface for Contexts, available or failed.
+    /// </summary>
+    public interface IContext : IIdentifiable
+    {
+        /// <summary>
+        /// the identifier of the Evaluator this EvaluatorContext is instantiated on.
+        /// </summary>
+        string EvaluatorId { get; set; }
+
+        /// <summary>
+        /// ID of the parent context, if there is any.
+        /// </summary>
+        Optional<string> ParentId { get; set; }
+
+        /// <summary>
+        /// descriptor of the Evaluator this Context is on.
+        /// </summary>
+        IEvaluatorDescriptor EvaluatorDescriptor { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/IFailedContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/IFailedContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IFailedContext.cs
new file mode 100644
index 0000000..7f9b94e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/IFailedContext.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.
+ */
+
+using Org.Apache.Reef.Utilities;
+
+namespace Org.Apache.Reef.Driver.Context
+{
+    public interface IFailedContext : IContext
+    {
+         Optional<IActiveContext> ParentContext { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextMessageSource.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextMessageSource.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextMessageSource.cs
new file mode 100644
index 0000000..9b4f3a3
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextMessageSource.cs
@@ -0,0 +1,42 @@
+/**
+ * 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.Utilities;
+
+namespace Org.Apache.Reef.Driver.Context
+{
+    /// <summary>
+    /// Default ContextMessageSource: return nothing.
+    /// </summary>
+    public class DefaultContextMessageSource : IContextMessageSource
+    {
+        public Optional<Common.Context.ContextMessage> Message
+        {
+            get
+            {
+                return Optional<Common.Context.ContextMessage>.Empty();
+            }
+
+            set
+            {
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStartHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStartHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStartHandler.cs
new file mode 100644
index 0000000..94af51e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStartHandler.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 Org.Apache.Reef.Common.Events;
+using Org.Apache.Reef.Utilities.Logging;
+using System;
+
+namespace Org.Apache.Reef.Driver.Context
+{
+   /// <summary>
+    /// Default handler for ContextStart
+   /// </summary>
+    public class DefaultContextStartHandler : IObserver<IContextStart>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultContextStartHandler));
+
+        public void OnNext(IContextStart contextStart)
+        {
+            LOGGER.Log(Level.Info, "DefaultContextStartHandler received for context: " + contextStart.Id);
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStopHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStopHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStopHandler.cs
new file mode 100644
index 0000000..a7f0220
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/context/defaults/DefaultContextStopHandler.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 Org.Apache.Reef.Common.Events;
+using Org.Apache.Reef.Utilities.Logging;
+using System;
+
+namespace Org.Apache.Reef.Driver.Context
+{
+    /// <summary>
+    /// Default event handler for ContextStop
+    /// </summary>
+    public class DefaultContextStopHandler : IObserver<IContextStop>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultContextStopHandler));
+
+        public void OnNext(IContextStop contextStop)
+        {
+            LOGGER.Log(Level.Info, "DefaultContextStopHandler received for context: " + contextStop.Id);
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/contract/IBridgeContract.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/contract/IBridgeContract.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/contract/IBridgeContract.cs
new file mode 100644
index 0000000..424fdb8
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/contract/IBridgeContract.cs
@@ -0,0 +1,26 @@
+/**
+ * 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.Contract
+{
+    public interface IBridgeContract
+    {
+        string InstanceId { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseHandler.cs
new file mode 100644
index 0000000..a69200f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseHandler.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.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default handler for close messages from the client: logging it
+    /// </summary>
+    public class DefaultClientCloseHandler : IObserver<byte[]>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultClientCloseHandler));
+
+        [Inject]
+        public DefaultClientCloseHandler()
+        {
+        }
+
+        public void OnNext(byte[] value)
+        {
+            LOGGER.Log(Level.Info, "Closing the Client");
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseWithMessageHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseWithMessageHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseWithMessageHandler.cs
new file mode 100644
index 0000000..16e004c
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientCloseWithMessageHandler.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 Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default handler for close messages from the client: Throw an Exception.
+    /// </summary>
+    public class DefaultClientCloseWithMessageHandler : IObserver<byte[]>
+    {
+        [Inject]
+        public DefaultClientCloseWithMessageHandler()
+        {
+        }
+        
+        public void OnNext(byte[] value)
+        {
+            Exceptions.Throw(new InvalidOperationException("No handler bound for client Close With Message event:" + ByteUtilities.ByteArrarysToString(value)), 
+                Logger.GetLogger(typeof(DefaultClientCloseWithMessageHandler)));
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientMessageHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientMessageHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientMessageHandler.cs
new file mode 100644
index 0000000..61689d0
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultClientMessageHandler.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 Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// DDefault event handler for Client messages: Logging it.
+    /// </summary>
+    public class DefaultClientMessageHandler : IObserver<byte[]>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultClientMessageHandler));
+        
+        [Inject]
+        public DefaultClientMessageHandler()
+        {
+        }
+
+        public void OnNext(byte[] value)
+        {
+            LOGGER.Log(Level.Info, "Received message: " + ByteUtilities.ByteArrarysToString(value));
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextActiveHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextActiveHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextActiveHandler.cs
new file mode 100644
index 0000000..bc0b482
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextActiveHandler.cs
@@ -0,0 +1,56 @@
+/**
+ * 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.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default handler for ActiveContext: Close it.
+    /// </summary>
+    public class DefaultContextActiveHandler : IObserver<IActiveContext>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultContextActiveHandler));
+        
+        [Inject]
+        public DefaultContextActiveHandler()
+        {
+        }
+
+        public void OnNext(IActiveContext value)
+        {
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "Received ActiveContext :[{0}], closing it", value.Id));
+            value.Dispose();
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextClosureHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextClosureHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextClosureHandler.cs
new file mode 100644
index 0000000..62e8966
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextClosureHandler.cs
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Context;
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default event handler for ClosedContext: Logging it.
+    /// </summary>
+    public class DefaultContextClosureHandler : IObserver<IClosedContext>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultContextClosureHandler));
+        
+        [Inject]
+        public DefaultContextClosureHandler()
+        {
+        }
+
+        public void OnNext(IClosedContext value)
+        {
+            LOGGER.Log(Level.Info, "Received ClosedContext :" + value.Id);
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextFailureHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextFailureHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextFailureHandler.cs
new file mode 100644
index 0000000..cb1621b
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextFailureHandler.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.Driver.Context;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    ///  Default event handler used for FailedContext: It crashes the driver.
+    /// </summary>
+    public class DefaultContextFailureHandler : IObserver<IFailedContext>
+    {
+        [Inject]
+        public DefaultContextFailureHandler()
+        {
+        }
+
+        public void OnNext(IFailedContext value)
+        {
+            throw new InvalidOperationException("No handler bound for FailedContext: " + value.Id);
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextMessageHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextMessageHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextMessageHandler.cs
new file mode 100644
index 0000000..f5b7ad2
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultContextMessageHandler.cs
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Context;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// efault event handler for ContextMessage: Logging it.
+    /// </summary>
+    public class DefaultContextMessageHandler : IObserver<IContextMessage>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultContextMessageHandler));
+
+        [Inject]
+        public DefaultContextMessageHandler()
+        {
+        }
+
+        public void OnNext(IContextMessage value)
+        {
+            LOGGER.Log(Level.Info, "Received ContextMessage: " + ByteUtilities.ByteArrarysToString(value.Message));
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultCustomTraceListener.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultCustomTraceListener.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultCustomTraceListener.cs
new file mode 100644
index 0000000..f197298
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultCustomTraceListener.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.Tang.Annotations;
+using System.Diagnostics;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    public class DefaultCustomTraceListener : TraceListener
+    {
+        private readonly TraceListener _listener; 
+
+        [Inject]
+        public DefaultCustomTraceListener()
+        {
+            _listener = new ConsoleTraceListener();
+        }
+
+        public override void Write(string message)
+        {
+            _listener.Write(message);
+        }
+
+        public override void WriteLine(string message)
+        {
+            _listener.WriteLine(message);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartContextActiveHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartContextActiveHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartContextActiveHandler.cs
new file mode 100644
index 0000000..314a132
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartContextActiveHandler.cs
@@ -0,0 +1,56 @@
+/**
+ * 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.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default handler for ActiveContext received during driver restart: Close it.
+    /// </summary>
+    public class DefaultDriverRestartContextActiveHandler : IObserver<IActiveContext>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultDriverRestartContextActiveHandler));
+        
+        [Inject]
+        public DefaultDriverRestartContextActiveHandler()
+        {
+        }
+
+        public void OnNext(IActiveContext value)
+        {
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "Received ActiveContext during driver restart:[{0}], closing it", value.Id));
+            value.Dispose();
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartHandler.cs
new file mode 100644
index 0000000..3603f61
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartHandler.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 Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Wake.Time;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    ///  Default event handler for driver restart: Logging it.
+    /// </summary>
+    public class DefaultDriverRestartHandler : IObserver<StartTime>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultDriverRestartHandler));
+        
+        [Inject]
+        public DefaultDriverRestartHandler()
+        {
+        }
+
+        public void OnNext(StartTime startTime)
+        {
+            LOGGER.Log(Level.Info, "Driver restarted at" + new DateTime(startTime.TimeStamp));
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartTaskRunningHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartTaskRunningHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartTaskRunningHandler.cs
new file mode 100644
index 0000000..461b3bb
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultDriverRestartTaskRunningHandler.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 Org.Apache.Reef.Driver.Task;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    ///  Default event handler for TaskRuntime received during driver restart: Logging it.
+    /// </summary>
+    public class DefaultDriverRestartTaskRunningHandler : IObserver<IRunningTask>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultDriverRestartTaskRunningHandler));
+        
+        [Inject]
+        public DefaultDriverRestartTaskRunningHandler()
+        {
+        }
+
+        public void OnNext(IRunningTask runningTask)
+        {
+            LOGGER.Log(Level.Info, "Received TaskRuntime during driver restart: " + runningTask.Id);
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorAllocationHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorAllocationHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorAllocationHandler.cs
new file mode 100644
index 0000000..07a5828
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorAllocationHandler.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.Evaluator;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default handler for AllocatedEvaluator: close it.
+    /// </summary>
+    public class DefaultEvaluatorAllocationHandler : IObserver<IAllocatedEvaluator>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultEvaluatorAllocationHandler));
+        
+        [Inject]
+        public DefaultEvaluatorAllocationHandler()
+        {
+        }
+
+        public void OnNext(IAllocatedEvaluator value)
+        {
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "Received AllocatedEvaluator : {0}, closing", value.Id));
+            value.Dispose();
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorCompletionHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorCompletionHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorCompletionHandler.cs
new file mode 100644
index 0000000..6297f64
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorCompletionHandler.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 Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default event handler for CompletedEvaluator: Logging it.
+    /// </summary>
+    public class DefaultEvaluatorCompletionHandler : IObserver<ICompletedEvaluator>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultEvaluatorCompletionHandler));
+        
+        [Inject]
+        public DefaultEvaluatorCompletionHandler()
+        {
+        }
+
+        public void OnNext(ICompletedEvaluator value)
+        {
+            LOGGER.Log(Level.Info, "Received CompletedEvaluator: " + value.Id);
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorFailureHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorFailureHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorFailureHandler.cs
new file mode 100644
index 0000000..ccd8d4c
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorFailureHandler.cs
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    ///  Default event handler used for FailedEvaluator: It crashes the driver.
+    /// </summary>
+    public class DefaultEvaluatorFailureHandler : IObserver<IFailedEvaluator>
+    {        
+        [Inject]
+        public DefaultEvaluatorFailureHandler()
+        {
+        }
+
+        public void OnNext(IFailedEvaluator value)
+        {
+            var e = new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, "Evaluator {0} failed, and no handler is bound for FailedEvaluator.", value.Id));
+            Exceptions.Throw(e, Logger.GetLogger(typeof(DefaultEvaluatorFailureHandler)));
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorRequestorHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorRequestorHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorRequestorHandler.cs
new file mode 100644
index 0000000..2b70281
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultEvaluatorRequestorHandler.cs
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Bridge;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default handler for close messages from the client: logging it
+    /// </summary>
+    public class DefaultEvaluatorRequestorHandler : IObserver<IEvaluatorRequestor>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultClientCloseHandler));
+
+        [Inject]
+        public DefaultEvaluatorRequestorHandler()
+        {
+        }
+
+        public void OnNext(IEvaluatorRequestor value)
+        {
+            LOGGER.Log(Level.Info, "Default evaluator requstor: requesting 1 evaluator with 512 MB");
+            int evaluatorsNumber = 1;
+            int memory = 512;
+            string rack = "WonderlandRack";
+            EvaluatorRequest request = new EvaluatorRequest(evaluatorsNumber, memory, rack);
+
+            value.Submit(request);
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultHttpHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultHttpHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultHttpHandler.cs
new file mode 100644
index 0000000..2dc874b
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultHttpHandler.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 System.Net;
+using Org.Apache.Reef.Driver.Bridge;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    public class DefaultHttpHandler : IHttpHandler
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultHttpHandler));
+
+        [Inject]
+        public DefaultHttpHandler()
+        {
+        }
+
+        public string GetSpecification()
+        {
+            return "Ping";
+        }
+
+        public void OnHttpRequest(ReefHttpRequest requet, ReefHttpResponse response) 
+        {
+            LOGGER.Log(Level.Info, "OnHttpRequest in DefaultHttpHandler is called.");
+            response.Status = HttpStatusCode.OK;
+            response.OutputStream = ByteUtilities.StringToByteArrays("Byte array returned from DefaultHttpHandler in CLR!!!");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskCompletionHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskCompletionHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskCompletionHandler.cs
new file mode 100644
index 0000000..5093c85
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskCompletionHandler.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 Org.Apache.Reef.Common.Context;
+using Org.Apache.Reef.Common.Task;
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Task;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// efault event handler for CompletedTask: Log it and close the context.
+    /// </summary>
+    public class DefaultTaskCompletionHandler : IObserver<ICompletedTask>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultTaskCompletionHandler));
+        
+        [Inject]
+        public DefaultTaskCompletionHandler()
+        {
+        }
+
+        public void OnNext(ICompletedTask value)
+        {
+            IActiveContext activeContext = value.ActiveContext;
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "Received CompletedTask: {0} :: CLOSING context: {1}", value.Id, activeContext.Id));
+            activeContext.Dispose();
+        }
+
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskFailureHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskFailureHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskFailureHandler.cs
new file mode 100644
index 0000000..99c7f8d
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/defaults/DefaultTaskFailureHandler.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.Common.Task;
+using Org.Apache.Reef.Driver.Task;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Driver.Defaults
+{
+    /// <summary>
+    /// Default event handler used for FailedTask: It crashes the driver.
+    /// </summary>
+    public class DefaultTaskFailureHandler : IObserver<IFailedTask>
+    {
+        [Inject]
+        public DefaultTaskFailureHandler()
+        {
+        }
+
+        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();
+        }
+    }
+}

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