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

[29/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/ReefCommon/catalog/IResourceCatalog.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/IResourceCatalog.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/IResourceCatalog.cs
new file mode 100644
index 0000000..0c77b95
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/IResourceCatalog.cs
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Capabilities;
+using System.Collections.Generic;
+
+namespace Org.Apache.Reef.Common.Catalog
+{
+    public interface IResourceCatalog
+    {
+        string Name { get; set; }
+
+        ICollection<ICapability> Capabilities { get; set; }
+
+        ICollection<INodeDescriptor> Nodes { get; set; }
+
+        ICollection<IRackDescriptor> Racks { get; set; }
+
+        INodeDescriptor GetNode(string nodeId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/NodeDescriptorImpl.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/NodeDescriptorImpl.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/NodeDescriptorImpl.cs
new file mode 100644
index 0000000..6e0fa67
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/NodeDescriptorImpl.cs
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Capabilities;
+using System.Collections.Generic;
+using System.Net;
+
+namespace Org.Apache.Reef.Common.Catalog
+{
+    public class NodeDescriptorImpl : INodeDescriptor
+    {
+        private RackDescriptorImpl _rack;
+
+        private string _id;
+
+        private IPEndPoint _address;
+
+        private RAM _ram;
+
+        private IList<ICapability> _capabilities;
+
+        public NodeDescriptorImpl()
+        {
+        }
+
+        public NodeDescriptorImpl(string id, IPEndPoint addresss, RackDescriptorImpl rack, RAM ram)
+        {
+            _id = id;
+            _address = addresss;
+            _rack = rack;
+            _ram = ram;
+            _capabilities = new List<ICapability>();
+            _rack.AddNodeDescriptor(this);
+        }
+
+        public RackDescriptorImpl Rack 
+        {
+            get
+            {
+                return _rack;
+            }
+        }
+
+        public string Id
+        {
+            get
+            {
+                return _id;
+            }
+        }
+
+        public string HostName { get; set; }
+
+        public CPU Cpu
+        {
+            get
+            {
+                return new CPU(1);
+            }
+
+            set
+            {
+            }
+        }
+
+        public RAM Ram
+        {
+            get
+            {
+                return _ram;
+            }
+
+            set
+            {
+                _ram = value;
+            }
+        }
+
+        public IList<ICapability> Capabilities
+        {
+            get
+            {
+                return _capabilities;
+            }
+        }
+
+        public IPEndPoint InetSocketAddress
+        {
+            get
+            {
+                return _address;
+            }
+
+            set
+            {
+                _address = value;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/RackDescriptorImpl.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/RackDescriptorImpl.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/RackDescriptorImpl.cs
new file mode 100644
index 0000000..3b28c98
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/RackDescriptorImpl.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.Common.Capabilities;
+using System;
+using System.Collections.Generic;
+using System.Text;
+
+namespace Org.Apache.Reef.Common.Catalog
+{
+    public class RackDescriptorImpl : IRackDescriptor
+    {
+        public RackDescriptorImpl(string name)
+        {
+            Name = name;
+            Capabilities = new List<ICapability>();
+            Nodes = new List<INodeDescriptor>();
+        }
+
+        public string Name { get; set; }
+
+        public ICollection<ICapability> Capabilities { get; set; }
+
+        public ICollection<INodeDescriptor> Nodes { get; set; }
+
+        public ICollection<IRackDescriptor> Racks { get; set; }
+
+        public INodeDescriptor GetNode(string nodeId)
+        {
+            throw new NotImplementedException();
+        }
+
+        public void AddNodeDescriptor(NodeDescriptorImpl node)
+        {
+            Nodes.Add(node);
+        }
+
+        public override string ToString()
+        {
+            StringBuilder stringBuilder = new StringBuilder();
+            stringBuilder.Append("Rack " + Name);
+            foreach (INodeDescriptor nodeDescriptor in Nodes)
+            {
+                stringBuilder.Append(Environment.NewLine + nodeDescriptor);
+            }
+            return stringBuilder.ToString();
+        }
+
+        public override int GetHashCode()
+        {
+            return Name.GetHashCode();
+        }
+
+        public override bool Equals(object obj)
+        {
+            return base.Equals(obj);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/ResourceCatalogImpl.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/ResourceCatalogImpl.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/ResourceCatalogImpl.cs
new file mode 100644
index 0000000..9348b67
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/ResourceCatalogImpl.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.Capabilities;
+using Org.Apache.Reef.Common.ProtoBuf.DriverRuntimeProto;
+using Org.Apache.Reef.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.Globalization;
+using System.Net;
+using System.Text;
+
+namespace Org.Apache.Reef.Common.Catalog
+{
+    public class ResourceCatalogImpl : IResourceCatalog
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(ResourceCatalogImpl));
+        
+        private static string defaultRackName = "default-rack";
+
+        private Dictionary<string, RackDescriptorImpl> _racks = new Dictionary<string, RackDescriptorImpl>();
+ 
+        private Dictionary<string, NodeDescriptorImpl> _nodes = new Dictionary<string, NodeDescriptorImpl>();
+
+        public string Name { get; set; }
+
+        public ICollection<ICapability> Capabilities { get; set; }
+
+        public ICollection<INodeDescriptor> Nodes { get; set; }
+
+        public ICollection<IRackDescriptor> Racks { get; set; }
+
+        public INodeDescriptor GetNode(string nodeId)
+        {
+            return _nodes[nodeId];
+        }
+
+        public void Handle(NodeDescriptorProto node)
+        {
+            string rackName = node.rack_name == null ? node.rack_name : defaultRackName;
+            string message = string.Format(
+                CultureInfo.InvariantCulture,
+                "Catalog new node: id[{0}], rack[{1}], host[{2}], port[{3}], memory[{4}]",
+                node.identifier,
+                rackName,
+                node.host_name,
+                node.port,
+                node.memory_size);
+            LOGGER.Log(Level.Info, message);
+            if (!string.IsNullOrWhiteSpace(rackName) && !_racks.ContainsKey(rackName))
+            {
+                RackDescriptorImpl newRack = new RackDescriptorImpl(rackName);
+                _racks.Add(rackName, newRack);
+            }
+            RackDescriptorImpl rack = _racks[rackName];
+            IPAddress ipAddress = null;
+            IPAddress.TryParse(node.host_name, out ipAddress);
+            if (ipAddress == null)
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new ArgumentException("cannot parse host ipaddress: " + node.host_name), LOGGER);
+            }
+            IPEndPoint ipEndPoint = new IPEndPoint(ipAddress, node.port);
+            RAM ram = new RAM(node.memory_size);
+            NodeDescriptorImpl nodeDescriptor = new NodeDescriptorImpl(node.identifier, ipEndPoint, rack, ram);
+            _nodes.Add(nodeDescriptor.Id, nodeDescriptor);
+        }
+
+        public override string ToString()
+        {
+            StringBuilder stringBuilder = new StringBuilder();
+            stringBuilder.Append("=== Resource Catalog ===");
+            foreach (IRackDescriptor rackDescriptor in Racks)
+            {
+                stringBuilder.Append(Environment.NewLine + rackDescriptor);
+            }
+            return stringBuilder.ToString();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/CPU.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/CPU.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/CPU.cs
new file mode 100644
index 0000000..0acda2d
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/CPU.cs
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using System.Globalization;
+using Org.Apache.Reef.Common.Api;
+using Org.Apache.Reef.Utilities.Logging;
+
+namespace Org.Apache.Reef.Common.Capabilities
+{
+    public class CPU : ICapability
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(CPU));
+
+        private int _cores;
+
+        public CPU(int cores)
+        {
+            if (cores <= 0)
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new ArgumentException("cores cannot be non-positive"), LOGGER);
+            }
+            _cores = cores;
+        }
+
+        public int Cores
+        {
+            get
+            {
+                return _cores;
+            }
+        }
+
+        public override string ToString()
+        {
+            return string.Format(CultureInfo.InvariantCulture, "CPU Cores = [{0}]", Cores);
+        }
+
+        public override int GetHashCode()
+        {
+            return Cores.GetHashCode();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/ICapability.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/ICapability.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/ICapability.cs
new file mode 100644
index 0000000..8761426
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/ICapability.cs
@@ -0,0 +1,25 @@
+/**
+ * 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.Common.Capabilities
+{
+    public interface ICapability
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/RAM.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/RAM.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/RAM.cs
new file mode 100644
index 0000000..ed6e078
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/capabilities/RAM.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 System.Globalization;
+
+namespace Org.Apache.Reef.Common.Capabilities
+{
+    public class RAM : ICapability
+    {
+        private int _megaBytes;
+
+        public RAM(int megaBytes)
+        {
+            _megaBytes = megaBytes;
+        }
+
+        public int MegaBytes
+        {
+            get
+            {
+                return _megaBytes;
+            }
+        }
+
+        public override string ToString()
+        {
+            return string.Format(CultureInfo.InvariantCulture, "RAM(in mega bytes) = [{0}]", MegaBytes);
+        }
+
+        public override int GetHashCode()
+        {
+            return MegaBytes.GetHashCode();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/context/ContextMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/context/ContextMessage.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/context/ContextMessage.cs
new file mode 100644
index 0000000..9c4737c
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/context/ContextMessage.cs
@@ -0,0 +1,66 @@
+/**
+ * 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.Utilities.Logging;
+
+namespace Org.Apache.Reef.Common.Context
+{
+    public class ContextMessage
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(ContextMessage));
+        private readonly string _messageSourcId;
+        private readonly byte[] _bytes;
+
+        private ContextMessage(string messageSourceId, byte[] bytes)
+        {
+            _messageSourcId = messageSourceId;
+            _bytes = bytes;
+        }
+
+        public string MessageSourceId
+        {
+            get { return _messageSourcId; }
+        }
+
+        public byte[] Bytes
+        {
+            get { return _bytes; }
+        }
+
+        /// <summary>
+        /// construt a new new ContextMessage with the given content.
+        /// </summary>
+        /// <param name="messageSourceId">The message's sourceID. This will be accessible in the Driver for routing.</param>
+        /// <param name="bytes">The actual content of the message, serialized into a byte[]</param>
+        /// <returns>new ContextMessage with the given content.</returns>
+        public static ContextMessage From(string messageSourceId, byte[] bytes)
+        {
+            if (string.IsNullOrEmpty(messageSourceId))
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new ArgumentNullException("messageSourceId"), LOGGER);
+            }
+            if (bytes == null)
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new ArgumentNullException("bytes"), LOGGER);
+            }
+            return new ContextMessage(messageSourceId, bytes);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessage.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessage.cs
new file mode 100644
index 0000000..183dafa
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessage.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.Common.Context
+{
+    public interface IContextMessage : IMessage, IIdentifiable
+    {
+        string MessageSourceId { get; }
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessageSource.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessageSource.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessageSource.cs
new file mode 100644
index 0000000..6025771
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/context/IContextMessageSource.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.Common.Context
+{
+    public interface IContextMessageSource
+    {
+        Optional<ContextMessage> Message { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultLocalHttpDriverConnection.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultLocalHttpDriverConnection.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultLocalHttpDriverConnection.cs
new file mode 100644
index 0000000..2e6cdc0
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultLocalHttpDriverConnection.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;
+
+namespace Org.Apache.Reef.Common.Evaluator
+{
+    public class DefaultLocalHttpDriverConnection : IDriverConnection
+    {
+        private readonly Uri _queryUri;
+
+        [Inject]
+        public DefaultLocalHttpDriverConnection()
+        {
+            _queryUri = new Uri(
+                    string.Concat(
+                    Constants.LocalHttpEndpointBaseUri,
+                    Constants.HttpReefUriSpecification,
+                    Constants.HttpDriverUriTarget));
+        }
+
+        public DriverInformation GetDriverInformation(string applicationId)
+        {
+            // application id not needed for local runtime
+            return DriverInformation.GetDriverInformationFromHttp(_queryUri);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnClusterHttpDriverConnection.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnClusterHttpDriverConnection.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnClusterHttpDriverConnection.cs
new file mode 100644
index 0000000..c09647d
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnClusterHttpDriverConnection.cs
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+namespace Org.Apache.Reef.Common.Evaluator
+{
+    public class DefaultYarnClusterHttpDriverConnection : IDriverConnection
+    {
+        [Inject]
+        public DefaultYarnClusterHttpDriverConnection()
+        {
+        }
+
+        public DriverInformation GetDriverInformation(string applicationId)
+        {
+            // e.g., http://headnodehost:9014/proxy/application_1407519727821_0012/reef/v1/driver
+            Uri queryUri = new Uri(
+                string.Concat(
+                Constants.HDInsightClusterHttpEndpointBaseUri,
+                applicationId,
+                Constants.HttpReefUriSpecification,
+                Constants.HttpDriverUriTarget));
+            return DriverInformation.GetDriverInformationFromHttp(queryUri);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnOneBoxHttpDriverConnection.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnOneBoxHttpDriverConnection.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnOneBoxHttpDriverConnection.cs
new file mode 100644
index 0000000..c12984e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DefaultYarnOneBoxHttpDriverConnection.cs
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Common.Evaluator
+{
+    public class DefaultYarnOneBoxHttpDriverConnection : IDriverConnection
+    {
+        [Inject]
+        public DefaultYarnOneBoxHttpDriverConnection()
+        {
+        }
+
+        public DriverInformation GetDriverInformation(string applicationId)
+        {
+            // e.g., http://yingdac1:8088/proxy/application_1407519727821_0012/reef/v1/driver
+            string oneBoxHost = string.Format(CultureInfo.InvariantCulture, "http://{0}:8088/proxy/", Environment.MachineName);
+            Uri queryUri = new Uri(
+                string.Concat(
+                oneBoxHost,
+                applicationId,
+                Constants.HttpReefUriSpecification,
+                Constants.HttpDriverUriTarget));
+            return DriverInformation.GetDriverInformationFromHttp(queryUri);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DriverInformation.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DriverInformation.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DriverInformation.cs
new file mode 100644
index 0000000..653627f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/DriverInformation.cs
@@ -0,0 +1,136 @@
+/**
+ * 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.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.Globalization;
+using System.IO;
+using System.Linq;
+using System.Net;
+using System.Text;
+
+namespace Org.Apache.Reef.Common.Evaluator
+{
+    public class DriverInformation
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DriverInformation));
+        
+        private string _rid;
+
+        private string _startTime;
+
+        private string _nameServerId;
+
+        private IList<AvroReefServiceInfo> _services;
+
+        public DriverInformation(string rid, string startTime, IList<AvroReefServiceInfo> services)
+        {
+            _rid = rid;
+            _startTime = startTime;
+            _services = services;
+
+            if (_services == null)
+            {
+                LOGGER.Log(Level.Warning, "no services information from driver.");
+            }
+            else
+            {
+                AvroReefServiceInfo nameServerInfo =
+                  _services.FirstOrDefault(
+                      s => s.serviceName.Equals(Constants.NameServerServiceName, StringComparison.OrdinalIgnoreCase));
+                if (nameServerInfo != null)
+                {
+                    _nameServerId = nameServerInfo.serviceInfo;
+                }
+            }  
+        }
+
+        public string DriverRemoteIdentifier
+        {
+            get
+            {
+                return _rid;
+            }
+        }
+
+        public string DriverStartTime
+        {
+            get
+            {
+                return _startTime;
+            }
+        }
+
+        public string NameServerId
+        {
+            get
+            {
+                return _nameServerId;
+            }
+        }
+
+        public static DriverInformation GetDriverInformationFromHttp(Uri queryUri)
+        {
+            HttpWebRequest request = (HttpWebRequest)WebRequest.Create(queryUri);
+            request.AllowAutoRedirect = false;
+            request.KeepAlive = false;
+            request.ContentType = "text/html";
+
+            string driverInfomation;
+            AvroDriverInfo info = null;
+            try
+            {
+                using (HttpWebResponse webResponse = (HttpWebResponse)request.GetResponse())
+                {
+                    Stream stream = webResponse.GetResponseStream();
+                    if (stream == null)
+                    {
+                        return null;
+                    }
+                    using (StreamReader streamReader = new StreamReader(stream, Encoding.UTF8))
+                    {
+                        driverInfomation = streamReader.ReadToEnd();
+                        LOGGER.Log(Level.Verbose, "Http response line: " + driverInfomation);
+                        info = AvroJsonSerializer<AvroDriverInfo>.FromString(driverInfomation);
+                    }
+                }
+            }
+            catch (WebException)
+            {
+                LOGGER.Log(Level.Warning, string.Format(CultureInfo.InvariantCulture, "In RECOVERY mode, cannot connect to [{0}] for driver information, will try again later.", queryUri));
+                return null;
+            }
+            catch (Exception e)
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.CaughtAndThrow(e, Level.Error, string.Format(CultureInfo.InvariantCulture, "Cannot read content from {0}.", queryUri), LOGGER);
+            }
+
+            if (info != null)
+            {
+                LOGGER.Log(
+                    Level.Verbose, 
+                    string.Format(CultureInfo.InvariantCulture, "Driver information extracted with remote identier [{0}], start time [{1}], and servics [{2}]", info.remoteId, info.startTime, info.services));
+                return new DriverInformation(info.remoteId, info.startTime, info.services);
+            }
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorOperationState.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorOperationState.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorOperationState.cs
new file mode 100644
index 0000000..77c1e63
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorOperationState.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.
+ */
+
+namespace Org.Apache.Reef.Common.Evaluator
+{
+    public enum EvaluatorOperationState
+    {
+        /// <summary>
+        /// default state
+        /// </summary>
+        UNINITIATED = 0,
+
+        /// <summary>
+        /// normal operational state
+        /// </summary>
+        OPERATIONAL = 1,
+
+        /// <summary>
+        /// in the process of recovering 
+        /// </summary>
+        RECOVERY = 2
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorRuntimeState.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorRuntimeState.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorRuntimeState.cs
new file mode 100644
index 0000000..d405c25
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorRuntimeState.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.
+ */
+
+namespace Org.Apache.Reef.Common.evaluator
+{
+    public enum EvaluatorRuntimeState
+    {
+        /// <summary>
+        /// default state
+        /// </summary>
+        UNINITIATED = 0,
+
+        /// <summary>
+        /// normal operational state
+        /// </summary>
+        RUNNING = 1,
+
+        /// <summary>
+        /// in the process of recovering 
+        /// </summary>
+        RECOVERY = 2
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorType.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorType.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorType.cs
new file mode 100644
index 0000000..4de31c8
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/EvaluatorType.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.
+ */
+
+namespace Org.Apache.Reef.Common.Evaluator
+{
+    public enum EvaluatorType
+    {
+        /// <summary>
+        /// default  type
+        /// </summary>
+        UNDECIDED = 0,
+
+        /// <summary>
+        /// Indicates an Evaluator that runs on the JVM
+        /// </summary>
+        JVM = 1,
+
+        /// <summary>
+        /// Indicates an Evaluator that runs on the CLR
+        /// </summary>
+        CLR = 2
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/IDriverConnection.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/IDriverConnection.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/IDriverConnection.cs
new file mode 100644
index 0000000..7113246
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/evaluator/IDriverConnection.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.Common.Evaluator
+{
+    public interface IDriverConnection
+    {
+        DriverInformation GetDriverInformation(string applicationId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/events/IContextStart.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/events/IContextStart.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/events/IContextStart.cs
new file mode 100644
index 0000000..bdd236d
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/events/IContextStart.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.Common.Events
+{
+    public interface IContextStart
+    {
+        string Id { 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/ReefCommon/events/IContextStop.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/events/IContextStop.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/events/IContextStop.cs
new file mode 100644
index 0000000..90031fb
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/events/IContextStop.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.Common.Events
+{
+    public interface IContextStop
+    {
+        string Id { 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/ReefCommon/exceptions/EvaluatorException.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/exceptions/EvaluatorException.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/exceptions/EvaluatorException.cs
new file mode 100644
index 0000000..4b62f29
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/exceptions/EvaluatorException.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.Common.Task;
+using Org.Apache.Reef.Utilities;
+using System;
+
+namespace Org.Apache.Reef.Common.Exceptions
+{
+    public class EvaluatorException : System.Exception, IIdentifiable
+    {
+        private readonly string _evaluatorId;
+
+        public EvaluatorException(string evaluatorId)
+        {
+            _evaluatorId = evaluatorId;
+            RunningTask = null;
+        }
+
+        public EvaluatorException(string evaluatorId, string message, System.Exception cause)
+            : base(message, cause)
+        {
+            _evaluatorId = evaluatorId;
+            RunningTask = null;
+        }
+
+        public EvaluatorException(string evaluatorId, string message)
+            : this(evaluatorId, message, (IRunningTask)null)
+        {
+        }
+
+        public EvaluatorException(string evaluatorId, string message, IRunningTask runningTask)
+            : base(message)
+        {
+            _evaluatorId = evaluatorId;
+            RunningTask = runningTask;
+        }
+
+        public EvaluatorException(string evaluatorId, System.Exception cause)
+            : this(evaluatorId, cause, null)
+        {
+        }
+
+        public EvaluatorException(string evaluatorId, Exception cause, IRunningTask runningTask)
+            : base(string.Empty, cause)
+        {
+            _evaluatorId = evaluatorId;
+            RunningTask = runningTask;
+        }
+
+        public IRunningTask RunningTask { get; set; }
+
+        public string Id
+        {
+            get { return _evaluatorId; }
+            set { }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/exceptions/JobException.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/exceptions/JobException.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/exceptions/JobException.cs
new file mode 100644
index 0000000..3f98f5e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/exceptions/JobException.cs
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Utilities;
+using System;
+
+namespace Org.Apache.Reef.Common
+{
+    public class JobException : Exception, IIdentifiable
+    {
+        private string _jobId;
+
+        public JobException(string jobId)
+        {
+            _jobId = jobId;
+        }
+
+        public JobException(string jobId, string message, Exception cause) 
+            : base(message, cause)
+        {
+            _jobId = jobId;
+        }
+
+        public JobException(string jobId, string message)
+            : base(message)
+        {
+            _jobId = jobId;
+        }
+
+        public JobException(string jobId, Exception cause)
+            : base(string.Empty, cause)
+        {
+            _jobId = jobId;
+        }
+
+        public string Id
+        {
+            get { return _jobId; }
+            set { }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/io/INameClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/io/INameClient.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/io/INameClient.cs
new file mode 100644
index 0000000..047b048
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/io/INameClient.cs
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.Net;
+
+namespace Org.Apache.Reef.Common.io
+{
+    /// <summary>
+    /// Client for the Reef name service. 
+    /// Used to register, unregister, and lookup IP Addresses of known hosts.
+    /// </summary>
+    public interface INameClient : IDisposable
+    {
+        /// <summary>
+        /// Registers the identifier with the NameService.  
+        /// Overwrites the previous mapping if the identifier has already 
+        /// been registered.
+        /// </summary>
+        /// <param name="id">The key used to map the remote endpoint</param>
+        /// <param name="endpoint">The endpoint to map</param>
+        void Register(string id, IPEndPoint endpoint);
+
+        /// <summary>
+        /// Unregisters the remote identifier with the NameService
+        /// </summary>
+        /// <param name="id">The identifier to unregister</param>
+        void Unregister(string id);
+
+        /// <summary>
+        /// Looks up the IPEndpoint for the registered identifier.
+        /// </summary>
+        /// <param name="id">The identifier to look up</param>
+        /// <returns>The mapped IPEndpoint for the identifier, or null if
+        /// the identifier has not been registered with the NameService</returns>
+        IPEndPoint Lookup(string id);
+
+        /// <summary>
+        /// Looks up the IPEndpoint for each of the registered identifiers in the list.
+        /// </summary>
+        /// <param name="ids">The list of identifiers to look up</param>
+        /// <returns>The list of NameAssignments representing a pair of identifer
+        /// and mapped IPEndpoint for that identifier.  If any of the requested identifiers
+        /// are not registered with the NameService, their corresponding NameAssignment
+        /// IPEndpoint value will be null.</returns>
+        List<NameAssignment> Lookup(List<string> ids);
+
+        /// <summary>
+        /// Restart the name client in case of failure.
+        /// </summary>
+        /// <param name="serverEndpoint">The new server endpoint to connect to</param>
+        void Restart(IPEndPoint serverEndpoint);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/io/NameAssignment.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/io/NameAssignment.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/io/NameAssignment.cs
new file mode 100644
index 0000000..4fbbfb0
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/io/NameAssignment.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.Net;
+using System.Net.Sockets;
+
+namespace Org.Apache.Reef.Common.io
+{
+    /// <summary>
+    /// Tuple containing the string identifier and IPEndpoint.
+    /// Used by NameServer and NameClient
+    /// </summary>
+    public class NameAssignment
+    {
+        public NameAssignment(string id, IPEndPoint endpoint)
+        {
+            Identifier = id;
+            Endpoint = endpoint;
+        }
+
+        public NameAssignment(string id, string address, int port)
+        {
+            Identifier = id;
+            IPAddress ipAddress;
+            if (!IPAddress.TryParse(address, out ipAddress))
+            {
+                IPHostEntry hostEntry = Dns.GetHostEntry(address);
+                foreach (var ip in hostEntry.AddressList)
+                {
+                    if (ip.AddressFamily == AddressFamily.InterNetwork)
+                    {
+                        ipAddress = ip;
+                        break;
+                    }
+                }
+            }
+            Endpoint = new IPEndPoint(ipAddress, port);
+        }
+
+        public string Identifier { get; set; }
+
+        public IPEndPoint Endpoint { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfiguration.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfiguration.cs
new file mode 100644
index 0000000..192c98c
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfiguration.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 System.Diagnostics.CodeAnalysis;
+using Org.Apache.Reef.Tang.Formats;
+using Org.Apache.Reef.Tang.Util;
+
+namespace Org.Apache.Reef.Common.io
+{
+    public class NamingConfiguration : ConfigurationModuleBuilder
+    {
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly RequiredParameter<string> NameServerAddress = new RequiredParameter<string>();
+
+        [SuppressMessage("Microsoft.Security", "CA2104:Do not declare read only mutable reference types", Justification = "not applicable")]
+        public static readonly RequiredParameter<int> NameServerPort = new RequiredParameter<int>();
+
+        public static ConfigurationModule ConfigurationModule
+        {
+            get
+            {
+                return new NamingConfiguration()
+                    .BindNamedParameter(GenericType<NamingConfigurationOptions.NameServerAddress>.Class, NameServerAddress)
+                    .BindNamedParameter(GenericType<NamingConfigurationOptions.NameServerPort>.Class, NameServerPort)
+                    .Build();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfigurationOptions.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfigurationOptions.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfigurationOptions.cs
new file mode 100644
index 0000000..0d222d6
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/io/NamingConfigurationOptions.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.Tang.Annotations;
+
+namespace Org.Apache.Reef.Common.io
+{
+    public class NamingConfigurationOptions
+    {
+        [NamedParameter("IP address of NameServer")]
+        public class NameServerAddress : Name<string>
+        {
+        }
+
+        [NamedParameter("Port of NameServer")]
+        public class NameServerPort : Name<int>
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/packages.config b/lang/cs/Source/REEF/reef-common/ReefCommon/packages.config
new file mode 100644
index 0000000..88cf17b
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/packages.config
@@ -0,0 +1,26 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<packages>
+  <package id="Microsoft.Hadoop.Avro" version="1.4.0.0" targetFramework="net45" />
+  <package id="Newtonsoft.Json" version="6.0.8" targetFramework="net45" />
+  <package id="protobuf-net" version="2.0.0.668" targetFramework="net45" />
+  <package id="Rx-Core" version="2.2.5" targetFramework="net45" />
+  <package id="Rx-Interfaces" version="2.2.5" targetFramework="net45" />
+</packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/ClientRuntime.pb.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/ClientRuntime.pb.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/ClientRuntime.pb.cs
new file mode 100644
index 0000000..8dddaee
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/ClientRuntime.pb.cs
@@ -0,0 +1,147 @@
+//------------------------------------------------------------------------------
+// <auto-generated>
+//     This code was generated by a tool.
+//
+//     Changes to this file may cause incorrect behavior and will be lost if
+//     the code is regenerated.
+// </auto-generated>
+//------------------------------------------------------------------------------
+
+// Generated from: client_runtime.proto
+// Note: requires additional types generated from: reef_service_protos.proto
+
+using Org.Apache.Reef.Common.ProtoBuf.ReefServiceProto;
+
+namespace Org.Apache.Reef.Common.ProtoBuf.ClienRuntimeProto{
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"JobSubmissionProto")]
+  public partial class JobSubmissionProto : global::ProtoBuf.IExtensible
+  {
+    public JobSubmissionProto() {}
+    
+    private string _identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string identifier
+    {
+      get { return _identifier; }
+      set { _identifier = value; }
+    }
+    private string _remote_id;
+    [global::ProtoBuf.ProtoMember(2, IsRequired = true, Name=@"remote_id", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string remote_id
+    {
+      get { return _remote_id; }
+      set { _remote_id = value; }
+    }
+    private string _configuration;
+    [global::ProtoBuf.ProtoMember(5, IsRequired = true, Name=@"configuration", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string configuration
+    {
+      get { return _configuration; }
+      set { _configuration = value; }
+    }
+    private string _user_name;
+    [global::ProtoBuf.ProtoMember(6, IsRequired = true, Name=@"user_name", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string user_name
+    {
+      get { return _user_name; }
+      set { _user_name = value; }
+    }
+    private SIZE _driver_size = SIZE.SMALL;
+    [global::ProtoBuf.ProtoMember(7, IsRequired = false, Name=@"driver_size", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(SIZE.SMALL)]
+    public SIZE driver_size
+    {
+      get { return _driver_size; }
+      set { _driver_size = value; }
+    }
+    private int _driver_memory = default(int);
+    [global::ProtoBuf.ProtoMember(8, IsRequired = false, Name=@"driver_memory", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int driver_memory
+    {
+      get { return _driver_memory; }
+      set { _driver_memory = value; }
+    }
+    private int _priority = default(int);
+    [global::ProtoBuf.ProtoMember(9, IsRequired = false, Name=@"priority", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int priority
+    {
+      get { return _priority; }
+      set { _priority = value; }
+    }
+    private string _queue = "";
+    [global::ProtoBuf.ProtoMember(10, IsRequired = false, Name=@"queue", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    [global::System.ComponentModel.DefaultValue("")]
+    public string queue
+    {
+      get { return _queue; }
+      set { _queue = value; }
+    }
+    private readonly global::System.Collections.Generic.List<FileResourceProto> _global_file = new global::System.Collections.Generic.List<FileResourceProto>();
+    [global::ProtoBuf.ProtoMember(11, Name=@"global_file", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public global::System.Collections.Generic.List<FileResourceProto> global_file
+    {
+      get { return _global_file; }
+    }
+  
+    private readonly global::System.Collections.Generic.List<FileResourceProto> _local_File = new global::System.Collections.Generic.List<FileResourceProto>();
+    [global::ProtoBuf.ProtoMember(12, Name=@"local_File", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public global::System.Collections.Generic.List<FileResourceProto> local_File
+    {
+      get { return _local_File; }
+    }
+  
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"JobControlProto")]
+  public partial class JobControlProto : global::ProtoBuf.IExtensible
+  {
+    public JobControlProto() {}
+    
+    private string _identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string identifier
+    {
+      get { return _identifier; }
+      set { _identifier = value; }
+    }
+    private Signal _signal = Signal.SIG_TERMINATE;
+    [global::ProtoBuf.ProtoMember(2, IsRequired = false, Name=@"signal", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(Signal.SIG_TERMINATE)]
+    public Signal signal
+    {
+      get { return _signal; }
+      set { _signal = value; }
+    }
+    private byte[] _message = null;
+    [global::ProtoBuf.ProtoMember(3, IsRequired = false, Name=@"message", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    [global::System.ComponentModel.DefaultValue(null)]
+    public byte[] message
+    {
+      get { return _message; }
+      set { _message = value; }
+    }
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+    [global::ProtoBuf.ProtoContract(Name=@"Signal")]
+    public enum Signal
+    {
+            
+      [global::ProtoBuf.ProtoEnum(Name=@"SIG_TERMINATE", Value=1)]
+      SIG_TERMINATE = 1,
+            
+      [global::ProtoBuf.ProtoEnum(Name=@"SIG_SUSPEND", Value=2)]
+      SIG_SUSPEND = 2,
+            
+      [global::ProtoBuf.ProtoEnum(Name=@"SIG_RESUME", Value=3)]
+      SIG_RESUME = 3
+    }
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/DriverRuntime.pb.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/DriverRuntime.pb.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/DriverRuntime.pb.cs
new file mode 100644
index 0000000..00dffe7
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/protobuf/cs/DriverRuntime.pb.cs
@@ -0,0 +1,339 @@
+//------------------------------------------------------------------------------
+// <auto-generated>
+//     This code was generated by a tool.
+//
+//     Changes to this file may cause incorrect behavior and will be lost if
+//     the code is regenerated.
+// </auto-generated>
+//------------------------------------------------------------------------------
+
+// Generated from: driver_runtime.proto
+// Note: requires additional types generated from: reef_service_protos.proto
+
+using Org.Apache.Reef.Common.ProtoBuf.ReefServiceProto;
+
+namespace Org.Apache.Reef.Common.ProtoBuf.DriverRuntimeProto
+{
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"DriverProcessRegistrationProto")]
+  public partial class DriverProcessRegistrationProto : global::ProtoBuf.IExtensible
+  {
+    public DriverProcessRegistrationProto() {}
+    
+    private string _remote_identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"remote_identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string remote_identifier
+    {
+      get { return _remote_identifier; }
+      set { _remote_identifier = value; }
+    }
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"NodeDescriptorProto")]
+  public partial class NodeDescriptorProto : global::ProtoBuf.IExtensible
+  {
+    public NodeDescriptorProto() {}
+    
+    private string _identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string identifier
+    {
+      get { return _identifier; }
+      set { _identifier = value; }
+    }
+    private string _host_name;
+    [global::ProtoBuf.ProtoMember(2, IsRequired = true, Name=@"host_name", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string host_name
+    {
+      get { return _host_name; }
+      set { _host_name = value; }
+    }
+    private int _port;
+    [global::ProtoBuf.ProtoMember(3, IsRequired = true, Name=@"port", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    public int port
+    {
+      get { return _port; }
+      set { _port = value; }
+    }
+    private int _memory_size;
+    [global::ProtoBuf.ProtoMember(4, IsRequired = true, Name=@"memory_size", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    public int memory_size
+    {
+      get { return _memory_size; }
+      set { _memory_size = value; }
+    }
+    private string _rack_name = "";
+    [global::ProtoBuf.ProtoMember(5, IsRequired = false, Name=@"rack_name", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    [global::System.ComponentModel.DefaultValue("")]
+    public string rack_name
+    {
+      get { return _rack_name; }
+      set { _rack_name = value; }
+    }
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"ResourceAllocationProto")]
+  public partial class ResourceAllocationProto : global::ProtoBuf.IExtensible
+  {
+    public ResourceAllocationProto() {}
+    
+    private string _identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string identifier
+    {
+      get { return _identifier; }
+      set { _identifier = value; }
+    }
+    private int _resource_memory;
+    [global::ProtoBuf.ProtoMember(2, IsRequired = true, Name=@"resource_memory", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    public int resource_memory
+    {
+      get { return _resource_memory; }
+      set { _resource_memory = value; }
+    }
+    private string _node_id;
+    [global::ProtoBuf.ProtoMember(3, IsRequired = true, Name=@"node_id", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string node_id
+    {
+      get { return _node_id; }
+      set { _node_id = value; }
+    }
+    private int _virtual_cores = default(int);
+    [global::ProtoBuf.ProtoMember(4, IsRequired = false, Name=@"virtual_cores", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int virtual_cores
+    {
+      get { return _virtual_cores; }
+      set { _virtual_cores = value; }
+    }
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"ResourceStatusProto")]
+  public partial class ResourceStatusProto : global::ProtoBuf.IExtensible
+  {
+    public ResourceStatusProto() {}
+    
+    private string _identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string identifier
+    {
+      get { return _identifier; }
+      set { _identifier = value; }
+    }
+    private State _state;
+    [global::ProtoBuf.ProtoMember(2, IsRequired = true, Name=@"state", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    public State state
+    {
+      get { return _state; }
+      set { _state = value; }
+    }
+    private string _diagnostics = "";
+    [global::ProtoBuf.ProtoMember(3, IsRequired = false, Name=@"diagnostics", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    [global::System.ComponentModel.DefaultValue("")]
+    public string diagnostics
+    {
+      get { return _diagnostics; }
+      set { _diagnostics = value; }
+    }
+    private int _exit_code = default(int);
+    [global::ProtoBuf.ProtoMember(4, IsRequired = false, Name=@"exit_code", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int exit_code
+    {
+      get { return _exit_code; }
+      set { _exit_code = value; }
+    }
+    private bool _is_from_previous_driver = default(bool);
+    [global::ProtoBuf.ProtoMember(5, IsRequired = false, Name=@"is_from_previous_driver", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    [global::System.ComponentModel.DefaultValue(default(bool))]
+    public bool is_from_previous_driver
+    {
+      get { return _is_from_previous_driver; }
+      set { _is_from_previous_driver = value; }
+    }
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"RuntimeStatusProto")]
+  public partial class RuntimeStatusProto : global::ProtoBuf.IExtensible
+  {
+    public RuntimeStatusProto() {}
+    
+    private string _name;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"name", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string name
+    {
+      get { return _name; }
+      set { _name = value; }
+    }
+    private State _state;
+    [global::ProtoBuf.ProtoMember(2, IsRequired = true, Name=@"state", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    public State state
+    {
+      get { return _state; }
+      set { _state = value; }
+    }
+    private RuntimeErrorProto _error = null;
+    [global::ProtoBuf.ProtoMember(3, IsRequired = false, Name=@"error", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    [global::System.ComponentModel.DefaultValue(null)]
+    public RuntimeErrorProto error
+    {
+      get { return _error; }
+      set { _error = value; }
+    }
+    private int _outstanding_container_requests = default(int);
+    [global::ProtoBuf.ProtoMember(5, IsRequired = false, Name=@"outstanding_container_requests", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int outstanding_container_requests
+    {
+      get { return _outstanding_container_requests; }
+      set { _outstanding_container_requests = value; }
+    }
+    private readonly global::System.Collections.Generic.List<string> _container_allocation = new global::System.Collections.Generic.List<string>();
+    [global::ProtoBuf.ProtoMember(6, Name=@"container_allocation", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public global::System.Collections.Generic.List<string> container_allocation
+    {
+      get { return _container_allocation; }
+    }
+  
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"ResourceRequestProto")]
+  public partial class ResourceRequestProto : global::ProtoBuf.IExtensible
+  {
+    public ResourceRequestProto() {}
+    
+    private int _memory_size = default(int);
+    [global::ProtoBuf.ProtoMember(2, IsRequired = false, Name=@"memory_size", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int memory_size
+    {
+      get { return _memory_size; }
+      set { _memory_size = value; }
+    }
+    private int _priority = default(int);
+    [global::ProtoBuf.ProtoMember(3, IsRequired = false, Name=@"priority", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int priority
+    {
+      get { return _priority; }
+      set { _priority = value; }
+    }
+    private int _virtual_cores = default(int);
+    [global::ProtoBuf.ProtoMember(4, IsRequired = false, Name=@"virtual_cores", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    [global::System.ComponentModel.DefaultValue(default(int))]
+    public int virtual_cores
+    {
+      get { return _virtual_cores; }
+      set { _virtual_cores = value; }
+    }
+    private int _resource_count;
+    [global::ProtoBuf.ProtoMember(5, IsRequired = true, Name=@"resource_count", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    public int resource_count
+    {
+      get { return _resource_count; }
+      set { _resource_count = value; }
+    }
+    private readonly global::System.Collections.Generic.List<string> _node_name = new global::System.Collections.Generic.List<string>();
+    [global::ProtoBuf.ProtoMember(6, Name=@"node_name", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public global::System.Collections.Generic.List<string> node_name
+    {
+      get { return _node_name; }
+    }
+  
+    private readonly global::System.Collections.Generic.List<string> _rack_name = new global::System.Collections.Generic.List<string>();
+    [global::ProtoBuf.ProtoMember(7, Name=@"rack_name", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public global::System.Collections.Generic.List<string> rack_name
+    {
+      get { return _rack_name; }
+    }
+  
+    private bool _relax_locality = default(bool);
+    [global::ProtoBuf.ProtoMember(10, IsRequired = false, Name=@"relax_locality", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    [global::System.ComponentModel.DefaultValue(default(bool))]
+    public bool relax_locality
+    {
+      get { return _relax_locality; }
+      set { _relax_locality = value; }
+    }
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"ResourceReleaseProto")]
+  public partial class ResourceReleaseProto : global::ProtoBuf.IExtensible
+  {
+    public ResourceReleaseProto() {}
+    
+    private string _identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string identifier
+    {
+      get { return _identifier; }
+      set { _identifier = value; }
+    }
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+  [global::System.Serializable, global::ProtoBuf.ProtoContract(Name=@"ResourceLaunchProto")]
+  public partial class ResourceLaunchProto : global::ProtoBuf.IExtensible
+  {
+    public ResourceLaunchProto() {}
+    
+    private string _identifier;
+    [global::ProtoBuf.ProtoMember(1, IsRequired = true, Name=@"identifier", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string identifier
+    {
+      get { return _identifier; }
+      set { _identifier = value; }
+    }
+    private string _remote_id;
+    [global::ProtoBuf.ProtoMember(2, IsRequired = true, Name=@"remote_id", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string remote_id
+    {
+      get { return _remote_id; }
+      set { _remote_id = value; }
+    }
+    private string _evaluator_conf;
+    [global::ProtoBuf.ProtoMember(3, IsRequired = true, Name=@"evaluator_conf", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public string evaluator_conf
+    {
+      get { return _evaluator_conf; }
+      set { _evaluator_conf = value; }
+    }
+    private ProcessType _type;
+    [global::ProtoBuf.ProtoMember(4, IsRequired = true, Name=@"type", DataFormat = global::ProtoBuf.DataFormat.TwosComplement)]
+    public ProcessType type
+    {
+      get { return _type; }
+      set { _type = value; }
+    }
+    private readonly global::System.Collections.Generic.List<FileResourceProto> _file = new global::System.Collections.Generic.List<FileResourceProto>();
+    [global::ProtoBuf.ProtoMember(10, Name=@"file", DataFormat = global::ProtoBuf.DataFormat.Default)]
+    public global::System.Collections.Generic.List<FileResourceProto> file
+    {
+      get { return _file; }
+    }
+  
+    private global::ProtoBuf.IExtension extensionObject;
+    global::ProtoBuf.IExtension global::ProtoBuf.IExtensible.GetExtensionObject(bool createIfMissing)
+      { return global::ProtoBuf.Extensible.GetExtensionObject(ref extensionObject, createIfMissing); }
+  }
+  
+}
\ No newline at end of file