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:07 UTC

[23/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/DriverBridgeConfigurationOptions.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/DriverBridgeConfigurationOptions.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/DriverBridgeConfigurationOptions.cs
new file mode 100644
index 0000000..73b1475
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/DriverBridgeConfigurationOptions.cs
@@ -0,0 +1,154 @@
+/**
+ * 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.Driver.Defaults;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Driver.Task;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Wake.Time;
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Diagnostics.CodeAnalysis;
+
+[module: SuppressMessage("StyleCop.CSharp.MaintainabilityRules", "SA1402:FileMayOnlyContainASingleClass", Justification = "allow name parameter class to be embedded")]
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    /// <summary>
+    /// Hosts all named parameters for Drivers, including bridge handlers.
+    /// </summary>
+    public class DriverBridgeConfigurationOptions
+    {
+        // Level.Verbose (since enum is not suppoted for TANG, we use a string here)
+        private const string _verboseLevel = "Verbose";
+
+        [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 evaluator is requested.", defaultClasses: new[] { typeof(DefaultEvaluatorRequestorHandler) })] 
+        public class EvaluatorRequestHandlers : Name<ISet<IObserver<IEvaluatorRequestor>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when an exception occurs on a running evaluator.", defaultClasses: new[] { typeof(DefaultEvaluatorFailureHandler) })]
+        public class FailedEvaluatorHandlers : Name<ISet<IObserver<IFailedEvaluator>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when an evaluator completes.", defaultClasses: new[] { typeof(DefaultEvaluatorCompletionHandler) })]
+        public class CompletedEvaluatorHandlers : Name<ISet<IObserver<ICompletedEvaluator>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Called when an allocated evaluator is given to the client.", defaultClasses: new[] { typeof(DefaultEvaluatorAllocationHandler) })]
+        public class AllocatedEvaluatorHandlers : Name<ISet<IObserver<IAllocatedEvaluator>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Running task handler.", defaultClasses: new[] { typeof(DefaultTaskRunningHandler) })]
+        public class RunningTaskHandlers : Name<ISet<IObserver<IRunningTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Running task during driver restart handler.", defaultClasses: new[] { typeof(DefaultDriverRestartTaskRunningHandler) })]
+        public class DriverRestartRunningTaskHandlers : Name<ISet<IObserver<IRunningTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Task exception handler.", defaultClasses: new[] { typeof(DefaultTaskFailureHandler) })]
+        public class FailedTaskHandlers : Name<ISet<IObserver<IFailedTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Task message handler.", defaultClasses: new[] { typeof(DefaultTaskMessageHandler) })]
+        public class TaskMessageHandlers : Name<ISet<IObserver<ITaskMessage>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Http Event Handlers.", defaultClasses: new[] { typeof(DefaultHttpHandler) })]
+        public class HttpEventHandlers : Name<ISet<IHttpHandler>>
+        {
+        }
+
+        [NamedParameter(documentation: "Completed task handler.", defaultClasses: new[] { typeof(DefaultTaskCompletionHandler) })]
+        public class CompletedTaskHandlers : Name<ISet<IObserver<ICompletedTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Suspended task handler.", defaultClasses: new[] { typeof(DefaultTaskSuspensionHandler) })]
+        public class SuspendedTaskHandlers : Name<ISet<IObserver<ISuspendedTask>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for IActiveContext.", defaultClasses: new[] { typeof(DefaultContextActiveHandler) })]
+        public class ActiveContextHandlers : Name<ISet<IObserver<IActiveContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for IActiveContext received during driver restart.", defaultClasses: new[] { typeof(DefaultDriverRestartContextActiveHandler) })]
+        public class DriverRestartActiveContextHandlers : Name<ISet<IObserver<IActiveContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for ClosedContext.", defaultClasses: new[] { typeof(DefaultContextClosureHandler) })]
+        public class ClosedContextHandlers : Name<ISet<IObserver<IClosedContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for FailedContext.", defaultClasses: new[] { typeof(DefaultContextFailureHandler) })]
+        public class FailedContextHandlers : Name<ISet<IObserver<IFailedContext>>>
+        {
+        }
+
+        [NamedParameter(documentation: "Handler for ContextMessage.", defaultClasses: new[] { typeof(DefaultContextMessageHandler) })]
+        public class ContextMessageHandlers : Name<ISet<IObserver<IContextMessage>>>
+        {
+        }
+
+        [NamedParameter("Command Line Arguments supplied by client", "CommandLineArguments", null)]
+        public class ArgumentSets : Name<ISet<string>>
+        {
+        }
+
+        [NamedParameter("Additional trace listners supplied by client", "TraceListeners", null, defaultClasses: new[] { typeof(DefaultCustomTraceListener) })]
+        public class TraceListenersSet : Name<ISet<TraceListener>>
+        {
+        }
+
+        [NamedParameter("Custom Trace Level", "TraceLevel", defaultValue: _verboseLevel)]
+        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/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpMessage.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpMessage.cs
new file mode 100644
index 0000000..bf10b5f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpMessage.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.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    [DataContract]
+    public class HttpMessage : IHttpMessage
+    {
+        public HttpMessage(IHttpServerBridgeClr2Java httpServerBridgeClr2Java)
+        {
+            HttpServerBridgeClr2Java = httpServerBridgeClr2Java;
+        }
+
+        [DataMember]
+        private IHttpServerBridgeClr2Java HttpServerBridgeClr2Java { get; set; }
+
+        public string GetRequestString()
+        {
+            return HttpServerBridgeClr2Java.GetQueryString();
+        }
+
+        public void SetQueryResult(string responseString)
+        {
+            HttpServerBridgeClr2Java.SetQueryResult(responseString);
+        }
+
+        public byte[] GetQueryReuestData()
+        {
+            return HttpServerBridgeClr2Java.GetQueryRequestData();            
+        }
+
+        public void SetQueryResponseData(byte[] responseData)
+        {
+            HttpServerBridgeClr2Java.SetQueryResponseData(responseData);
+        }
+
+        public void SetUriSpecification(string uriSpecification)
+        {
+            HttpServerBridgeClr2Java.SetUriSpecification(uriSpecification);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpServerHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpServerHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpServerHandler.cs
new file mode 100644
index 0000000..0609fb6
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpServerHandler.cs
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Avro;
+using Org.Apache.Reef.Driver.bridge;
+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.Globalization;
+using System.Linq;
+using System.Net;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    /// <summary>
+    ///  HttpServerHandler, the handler for all CLR http events
+    /// </summary>
+    public class HttpServerHandler : IObserver<IHttpMessage>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(HttpServerHandler));
+
+        private static readonly string SPEC = "SPEC";
+
+        private IDictionary<string, IHttpHandler> eventHandlers = new Dictionary<string, IHttpHandler>();
+
+        private HttpServerPort httpServerPort;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="HttpServerHandler" /> class.
+        /// </summary>
+        /// <param name="httpEventHandlers">The HTTP event handlers.</param>
+        /// <param name="httpServerPort">The HTTP server port.</param>
+        [Inject]
+        public HttpServerHandler([Parameter(Value = typeof(DriverBridgeConfigurationOptions.HttpEventHandlers))] ISet<IHttpHandler> httpEventHandlers,
+                                 HttpServerPort httpServerPort)
+        {
+            LOGGER.Log(Level.Info, "Constructing HttpServerHandler");       
+            foreach (var h in httpEventHandlers)
+            {
+                string spec = h.GetSpecification();
+                if (spec.Contains(":"))
+                {
+                    Exceptions.Throw(new ArgumentException("spec cannot contain :"), "The http spec given is " + spec, LOGGER);
+                }
+                LOGGER.Log(Level.Info, "HttpHandler spec:" + spec);   
+                eventHandlers.Add(spec.ToLower(CultureInfo.CurrentCulture), h);
+            }
+            this.httpServerPort = httpServerPort;
+        }
+
+        /// <summary>
+        /// Called when receving an http request from Java side
+        /// </summary>
+        /// <param name="httpMessage">The HTTP message.</param>
+        public void OnNext(IHttpMessage httpMessage)
+        {
+            LOGGER.Log(Level.Info, "HttpHandler OnNext is called");
+            string requestString = httpMessage.GetRequestString();
+
+            if (requestString != null && requestString.Equals(SPEC))
+            {
+                LOGGER.Log(Level.Info, "HttpHandler OnNext, requestString:" + requestString);
+                LOGGER.Log(Level.Info, "HttpHandler OnNext, port number:" + httpServerPort.PortNumber);
+
+                httpMessage.SetUriSpecification(GetAllSpecifications());
+            }
+            else
+            {
+                LOGGER.Log(Level.Info, "HttpHandler OnNext, handling http request.");
+                byte[] byteData = httpMessage.GetQueryReuestData();                    
+                AvroHttpRequest avroHttpRequest = AvroHttpSerializer.FromBytes(byteData);
+                LOGGER.Log(Level.Info, "HttpHandler OnNext, requestData:" + avroHttpRequest);
+
+                string spec = GetSpecification(avroHttpRequest.PathInfo);
+                if (spec != null)
+                {
+                    LOGGER.Log(Level.Info, "HttpHandler OnNext, target:" + spec);
+                    ReefHttpRequest request = ToHttpRequest(avroHttpRequest);
+                    ReefHttpResponse response = new ReefHttpResponse();
+
+                    IHttpHandler handler;
+                    eventHandlers.TryGetValue(spec.ToLower(CultureInfo.CurrentCulture), out handler);
+
+                    byte[] responseData;
+                    if (handler != null)
+                    {
+                        LOGGER.Log(Level.Info, "HttpHandler OnNext, get eventHandler:" + handler.GetSpecification());
+                        handler.OnHttpRequest(request, response);
+                        responseData = response.OutputStream;
+                    }
+                    else
+                    {
+                        responseData =
+                            ByteUtilities.StringToByteArrays(string.Format(CultureInfo.CurrentCulture,
+                                                                           "No event handler found at CLR side for {0}.",
+                                                                           spec));
+                    }
+                    httpMessage.SetQueryResponseData(responseData);
+                }
+            }
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnError(Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        private string GetAllSpecifications()
+        {
+            return string.Join(":", eventHandlers.Keys.ToArray());
+        }
+
+        private string GetSpecification(string requestUri)
+        {
+            if (requestUri != null)
+            {
+                string[] parts = requestUri.Split('/');
+
+                if (parts.Length > 1)
+                {
+                    return parts[1];
+                }
+            }
+            return null;            
+        }
+
+        private ReefHttpRequest ToHttpRequest(AvroHttpRequest avroRequest)
+        {
+            ReefHttpRequest httpRequest = new ReefHttpRequest();
+            httpRequest.PathInfo = avroRequest.PathInfo;
+            httpRequest.InputStream = avroRequest.InputStream;
+            httpRequest.Url = avroRequest.RequestUrl;
+            httpRequest.Querystring = avroRequest.QueryString;
+
+            HttpMethod m;
+            HttpMethod.TryParse(avroRequest.HttpMethod, true, out m);
+            httpRequest.Method = m;
+            return httpRequest;
+        }    
+    }
+}
\ 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/bridge/HttpServerPort.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpServerPort.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpServerPort.cs
new file mode 100644
index 0000000..af001e2
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/HttpServerPort.cs
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Driver.bridge
+{
+    public class HttpServerPort
+    {
+        [Inject]
+        public HttpServerPort()
+        {            
+        }
+
+        public int PortNumber { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpHandler.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpHandler.cs
new file mode 100644
index 0000000..4bd5895
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpHandler.cs
@@ -0,0 +1,39 @@
+/**
+ * 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;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface IHttpHandler
+    {
+        /// <summary>
+        /// Define the specification of the handler. ":" is not allowed in the specification.
+        /// </summary>
+        /// <returns>string specification</returns>
+        string GetSpecification();
+
+        /// <summary>
+        /// Called when Http request is sent
+        /// </summary>
+        /// <param name="requet">The requet.</param>
+        /// <param name="resonse">The resonse.</param>
+        void OnHttpRequest(ReefHttpRequest requet, ReefHttpResponse resonse);  
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpMessage.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpMessage.cs
new file mode 100644
index 0000000..59e66a6
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/IHttpMessage.cs
@@ -0,0 +1,33 @@
+/**
+ * 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.Bridge
+{
+    public interface IHttpMessage
+    {
+        string GetRequestString();
+        
+        void SetQueryResult(string responseString);
+
+        byte[] GetQueryReuestData();
+
+        void SetQueryResponseData(byte[] responseData);
+        
+        void SetUriSpecification(string uriSpecification);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/ReefHttpRequest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/ReefHttpRequest.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/ReefHttpRequest.cs
new file mode 100644
index 0000000..ccc9571
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/ReefHttpRequest.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.Globalization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public enum HttpMethod
+    {
+        Post,
+        Put,
+        Get,
+        Delete
+    }
+
+    public class ReefHttpRequest
+    {
+        public HttpMethod Method { get; set; }
+
+        public string Url { get; set; }
+
+        public string Querystring { get; set; }
+
+        public byte[] InputStream { get; set; }
+
+        public string PathInfo { get; set; }
+
+        public string Tostring()
+        {
+            return string.Format(CultureInfo.InvariantCulture, "Url: {0}, query string {1}", Url, Querystring);
+        }
+    }
+}
\ 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/bridge/ReefHttpResponse.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/ReefHttpResponse.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/ReefHttpResponse.cs
new file mode 100644
index 0000000..4eb3e6f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/ReefHttpResponse.cs
@@ -0,0 +1,30 @@
+/**
+ * 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;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public class ReefHttpResponse
+    {
+        public byte[] OutputStream { get; set; }
+
+        public HttpStatusCode Status { get; set; }
+    }
+}
\ 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/bridge/clr2java/IActiveContextClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IActiveContextClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IActiveContextClr2Java.cs
new file mode 100644
index 0000000..a61d639
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IActiveContextClr2Java.cs
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Evaluator;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface IActiveContextClr2Java : IClr2Java
+    {
+        void SubmitTask(string taskConfigStr);
+        
+        void Close();
+
+        string GetId();
+
+        string GetEvaluatorId();
+
+        IEvaluatorDescriptor GetEvaluatorDescriptor();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IAllocatedEvaluaotrClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IAllocatedEvaluaotrClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IAllocatedEvaluaotrClr2Java.cs
new file mode 100644
index 0000000..01f14e1
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IAllocatedEvaluaotrClr2Java.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.Driver.Evaluator;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface IAllocatedEvaluaotrClr2Java : IClr2Java
+    {
+        void SubmitContextAndTask(string contextConfigStr, string taskConfigStr);
+
+        void SubmitContext(string contextConfigStr);
+
+        void SubmitContextAndService(string contextConfigStr, string serviceConfigStr);
+
+        void SubmitContextAndServiceAndTask(string contextConfigStr, string serviceConfigStr, string taskConfigStr);
+
+        void Close();
+
+        string GetId();
+
+        string GetNameServerInfo();
+
+        IEvaluatorDescriptor GetEvaluatorDescriptor();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClosedContextClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClosedContextClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClosedContextClr2Java.cs
new file mode 100644
index 0000000..ed45211
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClosedContextClr2Java.cs
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Evaluator;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface IClosedContextClr2Java : IClr2Java
+    {        
+        string GetId();
+
+        string GetEvaluatorId();
+
+        IEvaluatorDescriptor GetEvaluatorDescriptor();
+
+        IActiveContextClr2Java GetParentContext();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClr2Java.cs
new file mode 100644
index 0000000..3f6c5c5
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IClr2Java.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.Bridge
+{
+    public interface IClr2Java
+    {
+        void OnError(string message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedEvaluatorClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedEvaluatorClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedEvaluatorClr2Java.cs
new file mode 100644
index 0000000..3a27723
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedEvaluatorClr2Java.cs
@@ -0,0 +1,30 @@
+/**
+ * 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.Bridge
+{
+    public interface ICompletedEvaluatorClr2Java : IClr2Java
+    {
+        /// <summary>
+        /// evaluator id
+        /// </summary>
+        /// <returns>id of the completed evaluator</returns>
+        string GetId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedTaskClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedTaskClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedTaskClr2Java.cs
new file mode 100644
index 0000000..a69789b
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ICompletedTaskClr2Java.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.Bridge
+{
+    public interface ICompletedTaskClr2Java : IClr2Java
+    {
+        IActiveContextClr2Java GetActiveContext();
+        
+        string GetId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IContextMessageClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IContextMessageClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IContextMessageClr2Java.cs
new file mode 100644
index 0000000..f50dca3
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IContextMessageClr2Java.cs
@@ -0,0 +1,30 @@
+/**
+ * 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.Bridge
+{
+    public interface IContextMessageClr2Java : IClr2Java
+    {
+        byte[] Get();
+
+        string GetId();
+
+        string GetMessageSourceId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IEvaluatorRequestorClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IEvaluatorRequestorClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IEvaluatorRequestorClr2Java.cs
new file mode 100644
index 0000000..b1ee4d1
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IEvaluatorRequestorClr2Java.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.Driver.Evaluator;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface IEvaluatorRequestorClr2Java : IClr2Java
+    {
+        void Submit(IEvaluatorRequest evaluatorRequest);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedContextClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedContextClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedContextClr2Java.cs
new file mode 100644
index 0000000..cd9a157
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedContextClr2Java.cs
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Evaluator;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface IFailedContextClr2Java : IClr2Java
+    {        
+        string GetId();
+
+        string GetEvaluatorId();
+
+        string GetParentId();
+
+        IEvaluatorDescriptor GetEvaluatorDescriptor();
+
+        IActiveContextClr2Java GetParentContext();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedEvaluatorClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedEvaluatorClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedEvaluatorClr2Java.cs
new file mode 100644
index 0000000..33bb4a4
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedEvaluatorClr2Java.cs
@@ -0,0 +1,30 @@
+/**
+ * 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;
+
+namespace Org.Apache.Reef.Driver
+{
+    public interface IFailedEvaluatorClr2Java
+    {
+        IEvaluatorRequestorClr2Java GetEvaluatorRequestor();
+
+        string GetId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedTaskClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedTaskClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedTaskClr2Java.cs
new file mode 100644
index 0000000..70dc34e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IFailedTaskClr2Java.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.Bridge
+{
+    public interface IFailedTaskClr2Java : IClr2Java
+    {
+        IActiveContextClr2Java GetActiveContext();
+
+        string GetString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IHttpServerBridgeClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IHttpServerBridgeClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IHttpServerBridgeClr2Java.cs
new file mode 100644
index 0000000..c30e3b4
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IHttpServerBridgeClr2Java.cs
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface IHttpServerBridgeClr2Java : IClr2Java
+    {
+        string GetQueryString();
+
+        void SetQueryResult(string queryResult);
+
+        byte[] GetQueryRequestData();
+
+        void SetQueryResponseData(byte[] responseData);
+
+        void SetUriSpecification(string uriSpecification);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IRunningTaskClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IRunningTaskClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IRunningTaskClr2Java.cs
new file mode 100644
index 0000000..eb95c31
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/IRunningTaskClr2Java.cs
@@ -0,0 +1,30 @@
+/**
+ * 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.Bridge
+{
+    public interface IRunningTaskClr2Java : IClr2Java
+    {
+        IActiveContextClr2Java GetActiveContext();
+
+        string GetId();
+
+        void Send(byte[] message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ISuspendedTaskClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ISuspendedTaskClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ISuspendedTaskClr2Java.cs
new file mode 100644
index 0000000..be04fd3
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ISuspendedTaskClr2Java.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.
+ */
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public interface ISuspendedTaskClr2Java : IClr2Java
+    {
+        /// <summary>
+        /// get active context the task is running in
+        /// </summary>
+        /// <returns>active context</returns>
+        IActiveContextClr2Java GetActiveContext();
+
+        /// <summary>
+        /// get suspsended task id
+        /// </summary>
+        /// <returns>suspsended task id</returns>
+        string GetId();
+
+        /// <summary>
+        /// get the message
+        /// </summary>
+        /// <returns>suspended task message</returns>
+        byte[] Get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ITaskMessageClr2Java.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ITaskMessageClr2Java.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ITaskMessageClr2Java.cs
new file mode 100644
index 0000000..c364587
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/clr2java/ITaskMessageClr2Java.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.Bridge
+{
+    public interface ITaskMessageClr2Java : IClr2Java
+    {
+        string GetId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ActiveContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ActiveContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ActiveContext.cs
new file mode 100644
index 0000000..fd7f826
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ActiveContext.cs
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Formats;
+using Org.Apache.Reef.Tang.Interface;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    [DataContract]
+    internal class ActiveContext : IActiveContext
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(ActiveContext));
+
+        private readonly AvroConfigurationSerializer _serializer;
+
+        public ActiveContext(IActiveContextClr2Java clr2Java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            Clr2Java = clr2Java;
+            _serializer = new AvroConfigurationSerializer();
+        }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        public string Id
+        {
+            get
+            {
+                return Clr2Java.GetId();
+            }
+
+            set
+            {
+            }
+        }
+
+        public string EvaluatorId
+        {
+            get
+            {
+                return Clr2Java.GetEvaluatorId();
+            }
+
+            set
+            {
+            }
+        }
+
+        public Optional<string> ParentId { get; set; }
+
+        public IEvaluatorDescriptor EvaluatorDescriptor
+        {
+            get
+            {
+                return Clr2Java.GetEvaluatorDescriptor();
+            }
+
+            set
+            {
+            }
+        }
+
+        private IActiveContextClr2Java Clr2Java { get; set; }
+
+        public void SubmitTask(IConfiguration taskConfiguration)
+        {
+            LOGGER.Log(Level.Info, "ActiveContext::SubmitTask");
+            string task = _serializer.ToString(taskConfiguration);
+            LOGGER.Log(Level.Info, "serialized taskConfiguration: " + task);
+            Clr2Java.SubmitTask(task);
+        }
+
+        public void Dispose()
+        {
+            LOGGER.Log(Level.Info, "ActiveContext::Dispose");
+            Clr2Java.Close();
+        }
+
+        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/bridge/events/AllocatedEvaluator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/AllocatedEvaluator.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/AllocatedEvaluator.cs
new file mode 100644
index 0000000..8ef9928
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/AllocatedEvaluator.cs
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Catalog;
+using Org.Apache.Reef.Common.Evaluator;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Formats;
+using Org.Apache.Reef.Tang.Interface;
+using System;
+using System.Collections.Generic;
+using System.Globalization;
+using System.Net;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    [DataContract]
+    internal class AllocatedEvaluator : IAllocatedEvaluator
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(AllocatedEvaluator));
+        
+        private readonly AvroConfigurationSerializer _serializer;
+
+        private IEvaluatorDescriptor _evaluatorDescriptor;
+
+        public AllocatedEvaluator(IAllocatedEvaluaotrClr2Java clr2Java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            _serializer = new AvroConfigurationSerializer();
+            Clr2Java = clr2Java;
+            Id = Clr2Java.GetId();
+            ProcessNewEvaluator();
+
+            NameServerInfo = Clr2Java.GetNameServerInfo();
+        }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        public string Id { get; set; }
+
+        public string EvaluatorBatchId { get; set; }
+
+        public EvaluatorType Type { get; set; }
+
+        public string NameServerInfo { get; set; }
+
+        [DataMember]
+        private IAllocatedEvaluaotrClr2Java Clr2Java { get; set; }
+
+        public void SubmitContext(IConfiguration contextConfiguration)
+        {
+            LOGGER.Log(Level.Info, "AllocatedEvaluator::SubmitContext");
+            string context = _serializer.ToString(contextConfiguration);
+            LOGGER.Log(Level.Info, "serialized contextConfiguration: " + context);
+            Clr2Java.SubmitContext(context);
+        }
+
+        public void SubmitContextAndTask(IConfiguration contextConfiguration, IConfiguration taskConfiguration)
+        {
+            LOGGER.Log(Level.Info, "AllocatedEvaluator::SubmitContextAndTask");
+
+            string context = _serializer.ToString(contextConfiguration);
+            string task = _serializer.ToString(taskConfiguration);
+
+            LOGGER.Log(Level.Info, "serialized contextConfiguration: " + context);
+            LOGGER.Log(Level.Info, "serialized taskConfiguration: " + task);
+
+            Clr2Java.SubmitContextAndTask(context, task);
+        }
+
+        public void SubmitContextAndService(IConfiguration contextConfiguration, IConfiguration serviceConfiguration)
+        {
+            LOGGER.Log(Level.Info, "AllocatedEvaluator::SubmitContextAndService");
+
+            string context = _serializer.ToString(contextConfiguration);
+            string service = _serializer.ToString(serviceConfiguration);
+
+            LOGGER.Log(Level.Info, "serialized contextConfiguration: " + context);
+            LOGGER.Log(Level.Info, "serialized serviceConfiguration: " + service);
+
+            Clr2Java.SubmitContextAndService(context, service);
+        }
+
+        public void SubmitContextAndServiceAndTask(IConfiguration contextConfiguration, IConfiguration serviceConfiguration, IConfiguration taskConfiguration)
+        {
+            LOGGER.Log(Level.Info, "AllocatedEvaluator::SubmitContextAndServiceAndTask");
+
+            string context = _serializer.ToString(contextConfiguration);
+            string service = _serializer.ToString(serviceConfiguration);
+            string task = _serializer.ToString(taskConfiguration);
+
+            LOGGER.Log(Level.Info, "serialized contextConfiguration: " + context);
+            LOGGER.Log(Level.Info, "serialized serviceConfiguration: " + service);
+            LOGGER.Log(Level.Info, "serialized taskConfiguration: " + task);
+
+            Clr2Java.SubmitContextAndServiceAndTask(context, service, task);
+        }
+
+        public IEvaluatorDescriptor GetEvaluatorDescriptor()
+        {
+            return _evaluatorDescriptor;
+        }
+
+        public void Dispose()
+        {
+            Clr2Java.Close();
+        }
+
+        public INodeDescriptor GetNodeDescriptor()
+        {
+            throw new NotImplementedException();
+        }
+
+        public void AddFile(string file)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void AddLibrary(string file)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void AddFileResource(string file)
+        {
+            throw new NotImplementedException();
+        }
+
+        private void ProcessNewEvaluator()
+        {
+            _evaluatorDescriptor = Clr2Java.GetEvaluatorDescriptor();
+            lock (EvaluatorRequestor.Evaluators)
+            {
+                foreach (KeyValuePair<string, IEvaluatorDescriptor> pair in EvaluatorRequestor.Evaluators)
+                {
+                    if (pair.Value.Equals(_evaluatorDescriptor))
+                    {
+                        string key = pair.Key;
+                        EvaluatorRequestor.Evaluators.Remove(key);
+                        string assignedId = key.Substring(0, key.LastIndexOf('_'));
+                        string message = string.Format(
+                            CultureInfo.InvariantCulture,
+                            "Received evalautor [{0}] of memory {1}MB that matches request of {2}MB with batch id [{3}].",
+                            Id,
+                            _evaluatorDescriptor.Memory,
+                            pair.Value.Memory,
+                            assignedId);
+
+                        LOGGER.Log(Level.Verbose, message);
+                        EvaluatorBatchId = assignedId;
+                        break;
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ClosedContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ClosedContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ClosedContext.cs
new file mode 100644
index 0000000..a643578
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ClosedContext.cs
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public class ClosedContext : IClosedContext
+    {
+        private string _id;
+
+        private string _evaluatorId;
+
+        public ClosedContext(IClosedContextClr2Java clr2java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            _id = clr2java.GetId();
+            _evaluatorId = clr2java.GetEvaluatorId();
+        }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        public string Id
+        {
+            get
+            {
+                return _id;
+            }
+
+            set
+            {
+            }
+        }
+
+        public string EvaluatorId
+        {
+            get
+            {
+                return _evaluatorId;
+            }
+
+            set
+            {
+            }
+        }
+
+        public Optional<string> ParentId { get; set; }
+
+        public IEvaluatorDescriptor EvaluatorDescriptor
+        {
+            get
+            {
+                return ClosedContextClr2JavaClr2Java.GetEvaluatorDescriptor();
+            }
+
+            set
+            {
+            }
+        }
+
+        public IActiveContext ParentContext
+        {
+            get
+            {
+                return new ActiveContext(ParentContextClr2Java);
+            }
+
+            set
+            {
+            }
+        }
+
+        private IActiveContextClr2Java ParentContextClr2Java { get; set; }
+
+        private IClosedContextClr2Java ClosedContextClr2JavaClr2Java { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedEvaluator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedEvaluator.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedEvaluator.cs
new file mode 100644
index 0000000..0815ee1
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedEvaluator.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.Driver.Evaluator;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    [DataContract]
+    internal class CompletedEvaluator : ICompletedEvaluator
+    {
+        private string _instanceId;
+
+        public CompletedEvaluator(ICompletedEvaluatorClr2Java clr2Java)
+        {
+            _instanceId = Guid.NewGuid().ToString("N");
+            CompletedEvaluatorClr2Java = clr2Java;
+        }
+
+        [DataMember]
+        public string InstanceId
+        {
+            get { return _instanceId; }
+            set { _instanceId = value; }
+        }
+
+        [DataMember]
+        public string Id
+        {
+            get
+            {
+                return CompletedEvaluatorClr2Java.GetId();
+            }
+
+            set
+            {
+            }
+        }
+
+        [DataMember]
+        public ICompletedEvaluatorClr2Java CompletedEvaluatorClr2Java { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedTask.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedTask.cs
new file mode 100644
index 0000000..30799db
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/CompletedTask.cs
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Task;
+using Org.Apache.Reef.Utilities.Logging;
+using System;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    [DataContract]
+    internal class CompletedTask : ICompletedTask
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(CompletedTask));
+
+        internal CompletedTask(ICompletedTaskClr2Java completedTaskClr2Java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            CompletedTaskClr2Java = completedTaskClr2Java;
+            ActiveContextClr2Java = completedTaskClr2Java.GetActiveContext();
+        }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        public byte[] Message { get; set; }
+
+        public string Id
+        {
+            get
+            {
+                return CompletedTaskClr2Java.GetId();
+            }
+
+            set
+            {
+            }
+        }
+
+        public IActiveContext ActiveContext
+        {
+            get
+            {
+                return new ActiveContext(ActiveContextClr2Java);
+            }
+
+            set
+            {
+            }
+        }
+
+        [DataMember]
+        private ICompletedTaskClr2Java CompletedTaskClr2Java { 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/ContextMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ContextMessage.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ContextMessage.cs
new file mode 100644
index 0000000..890cc77
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/ContextMessage.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.Context;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public class ContextMessage : IContextMessage
+    {
+        private readonly string _messageSourcId;
+        private readonly byte[] _bytes;
+        private readonly string _id;
+
+        public ContextMessage(IContextMessageClr2Java clr2Java)
+        {
+            _messageSourcId = clr2Java.GetMessageSourceId();
+            _bytes = clr2Java.Get();
+            _id = clr2Java.GetId();
+        }
+
+        public string Id
+        {
+            get { return _id; }
+            set { }
+        }
+
+        public string MessageSourceId
+        {
+            get { return _messageSourcId; }
+        }
+
+        public byte[] Message
+        {
+            get { return _bytes; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/EvaluatorRequstor.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/EvaluatorRequstor.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/EvaluatorRequstor.cs
new file mode 100644
index 0000000..3f7da32
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/EvaluatorRequstor.cs
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Catalog;
+using Org.Apache.Reef.Common.Evaluator;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.Globalization;
+using System.Runtime.Serialization;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    [DataContract]
+    internal class EvaluatorRequestor : IEvaluatorRequestor
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(EvaluatorRequestor));
+
+        private static Dictionary<string, IEvaluatorDescriptor> _evaluators;
+        
+        public EvaluatorRequestor(IEvaluatorRequestorClr2Java clr2Java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            Clr2Java = clr2Java;
+        }
+
+        public static Dictionary<string, IEvaluatorDescriptor> Evaluators
+        {
+            get
+            {
+                if (_evaluators == null)
+                {
+                    _evaluators = new Dictionary<string, IEvaluatorDescriptor>();
+                }
+                return _evaluators;
+            }
+        }
+
+        public IResourceCatalog ResourceCatalog { get; set; }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        [DataMember]
+        private IEvaluatorRequestorClr2Java Clr2Java { get; set; }
+
+        public void Submit(IEvaluatorRequest request)
+        {
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "Submitting request for {0} evaluators and {1} MB memory and  {2} core to rack {3}.", request.Number, request.MemoryMegaBytes, request.VirtualCore, request.Rack));
+
+            lock (Evaluators)
+            {
+                for (int i = 0; i < request.Number; i++)
+                {
+                    EvaluatorDescriptorImpl descriptor = new EvaluatorDescriptorImpl(new NodeDescriptorImpl(), EvaluatorType.CLR, request.MemoryMegaBytes, request.VirtualCore);
+                    descriptor.Rack = request.Rack;
+                    string key = string.Format(CultureInfo.InvariantCulture, "{0}_{1}", request.EvaluatorBatchId, i);
+                    try
+                    {
+                        _evaluators.Add(key, descriptor);
+                    }
+                    catch (ArgumentException e)
+                    {
+                        Exceptions.Caught(e, Level.Error, string.Format(CultureInfo.InvariantCulture, "EvaluatorBatchId [{0}] already exists.", key), LOGGER);
+                        Exceptions.Throw(new InvalidOperationException("Cannot use evaluator id " + key, e), LOGGER);
+                    }
+                }
+            }
+            
+            Clr2Java.Submit(request);
+        }
+
+        public void Dispose()
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedContext.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedContext.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedContext.cs
new file mode 100644
index 0000000..8f0a8e3
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedContext.cs
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Utilities;
+
+namespace Org.Apache.Reef.Driver.Bridge
+{
+    public class FailedContext : IFailedContext
+    {
+        private string _id;
+
+        private string _evaluatorId;
+
+        private string _parentId;
+
+        public FailedContext(IFailedContextClr2Java clr2Java)
+        {
+            _id = clr2Java.GetId();
+            _evaluatorId = clr2Java.GetEvaluatorId();
+            _parentId = clr2Java.GetParentId();
+            FailedContextClr2Java = clr2Java;
+        }
+
+        public string Id
+        {
+            get
+            {
+                return _id;
+            }
+
+            set
+            {
+            }
+        }
+
+        public string EvaluatorId
+        {
+            get
+            {
+                return _evaluatorId;
+            }
+
+            set
+            {
+            }
+        }
+
+        public Optional<string> ParentId
+        {
+            get
+            {
+                return string.IsNullOrEmpty(_parentId) ? 
+                    Optional<string>.Empty() : 
+                    Optional<string>.Of(_parentId);
+            }
+
+            set
+            {
+            }
+        }
+
+        public IEvaluatorDescriptor EvaluatorDescriptor
+        {
+            get
+            {
+                return FailedContextClr2Java.GetEvaluatorDescriptor();
+            }
+
+            set
+            {
+            }
+        }
+
+        public Optional<IActiveContext> ParentContext
+        {
+            get
+            {
+                IActiveContextClr2Java context = FailedContextClr2Java.GetParentContext();
+                if (context != null)
+                {
+                    return Optional<IActiveContext>.Of(new ActiveContext(context));
+                }
+                else
+                {
+                    return Optional<IActiveContext>.Empty();
+                }
+            }
+        }
+
+        private IFailedContextClr2Java FailedContextClr2Java { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedEvaluator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedEvaluator.cs b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedEvaluator.cs
new file mode 100644
index 0000000..8c12dee
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefDriver/bridge/events/FailedEvaluator.cs
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Exceptions;
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Evaluator;
+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
+{
+    [DataContract]
+    internal class FailedEvaluator : IFailedEvaluator
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(FailedEvaluator));
+
+        public FailedEvaluator(IFailedEvaluatorClr2Java clr2Java)
+        {
+            InstanceId = Guid.NewGuid().ToString("N");
+            FailedEvaluatorClr2Java = clr2Java;
+            EvaluatorRequestorClr2Java = FailedEvaluatorClr2Java.GetEvaluatorRequestor();
+            Id = FailedEvaluatorClr2Java.GetId();
+        }
+
+        [DataMember]
+        public string InstanceId { get; set; }
+
+        public string Id { get; set; }
+
+        public EvaluatorException EvaluatorException { get; set; }
+
+        public List<FailedContext> FailedContexts { get; set; }
+
+        public Optional<IFailedTask> FailedTask { get; set; }
+
+        [DataMember]
+        private IFailedEvaluatorClr2Java FailedEvaluatorClr2Java { get; set; }
+
+        [DataMember]
+        private IEvaluatorRequestorClr2Java EvaluatorRequestorClr2Java { get; set; }
+
+        public IEvaluatorRequestor GetEvaluatorRequetor()
+        {
+            if (EvaluatorRequestorClr2Java == null)
+            {
+                Exceptions.Throw(new InvalidOperationException("EvaluatorRequestorClr2Java not initialized."), LOGGER);
+            }
+            return new EvaluatorRequestor(EvaluatorRequestorClr2Java);
+        }
+    }
+}