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

[30/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-applications/Evaluator/Evaluator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-applications/Evaluator/Evaluator.cs b/lang/cs/Source/REEF/reef-applications/Evaluator/Evaluator.cs
new file mode 100644
index 0000000..0489154
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-applications/Evaluator/Evaluator.cs
@@ -0,0 +1,260 @@
+/**
+ * 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.Configuration;
+using System.Diagnostics;
+using System.Globalization;
+using System.IO;
+using System.Linq;
+using System.Threading;
+using System.Threading.Tasks;
+using Org.Apache.Reef.Common;
+using Org.Apache.Reef.Common.Context;
+using Org.Apache.Reef.Common.Evaluator.Context;
+using Org.Apache.Reef.Common.ProtoBuf.ReefProtocol;
+using Org.Apache.Reef.Driver.Bridge;
+using Org.Apache.Reef.Services;
+using Org.Apache.Reef.Tasks;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Formats;
+using Org.Apache.Reef.Tang.Implementations;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Wake.Remote;
+using Org.Apache.Reef.Wake.Remote.Impl;
+using Org.Apache.Reef.Wake.Time.Runtime.Event;
+
+namespace Org.Apache.Reef.Evaluator
+{
+    public class Evaluator
+    {
+        private static Logger _logger;
+
+        private static int _heartbeatPeriodInMs = Constants.DefaultEvaluatorHeartbeatPeriodInMs;
+
+        private static int _heartbeatMaxRetry = Constants.DefaultEvaluatorHeartbeatMaxRetry;
+
+        private static IInjector _injector;
+
+        private static EvaluatorConfigurations _evaluatorConfig;
+
+        public static void Main(string[] args)
+        {
+            Console.WriteLine(string.Format(CultureInfo.InvariantCulture, "START: {0} Evaluator::InitInjector.", DateTime.Now));
+            Stopwatch timer = new Stopwatch();
+            InitInjector();
+            SetCustomTraceListners();
+            timer.Stop();
+            Console.WriteLine(string.Format(CultureInfo.InvariantCulture, "EXIT: {0} Evaluator::InitInjector. Duration: [{1}].", DateTime.Now, timer.Elapsed));
+
+            RuntimeClock clock;
+
+            using (_logger.LogScope("Evaluator::Main"))
+            {
+                string debugEnabledString = Environment.GetEnvironmentVariable("Org.Apache.Reef.EvaluatorDebug");
+                if (!string.IsNullOrWhiteSpace(debugEnabledString) &&
+                    debugEnabledString.Equals("enabled", StringComparison.OrdinalIgnoreCase))
+                {
+                    while (true)
+                    {
+                        if (Debugger.IsAttached)
+                        {
+                            break;
+                        }
+                        else
+                        {
+                            _logger.Log(Level.Info, "Evaluator in debug mode, waiting for debugger to be attached...");
+                            Thread.Sleep(2000);
+                        }
+                    }
+                }
+
+                AppDomain.CurrentDomain.UnhandledException += UnhandledExceptionHandler;
+
+                string heartbeatPeriodFromConfig = ConfigurationManager.AppSettings["EvaluatorHeartbeatPeriodInMs"];
+
+                int heartbeatPeriod = 0;
+
+                if (!string.IsNullOrWhiteSpace(heartbeatPeriodFromConfig) &&
+                    int.TryParse(heartbeatPeriodFromConfig, out heartbeatPeriod))
+                {
+                    _heartbeatPeriodInMs = heartbeatPeriod;
+                }
+                _logger.Log(Level.Verbose,
+                            "Evaluator heartbeat period set to be " + _heartbeatPeriodInMs + " milliSeconds.");
+
+                int maxHeartbeatRetry = 0;
+                string heartbeatMaxRetryFromConfig = ConfigurationManager.AppSettings["EvaluatorHeartbeatRetryMaxTimes"];
+
+                if (!string.IsNullOrWhiteSpace(heartbeatMaxRetryFromConfig) &&
+                    int.TryParse(heartbeatMaxRetryFromConfig, out maxHeartbeatRetry))
+                {
+                    _heartbeatMaxRetry = maxHeartbeatRetry;
+                }
+                _logger.Log(Level.Verbose, "Evaluator heatrbeat max retry set to be " + _heartbeatMaxRetry + " times.");
+
+                if (args.Count() < 2)
+                {
+                    var e = new InvalidOperationException("must supply at least the rId and evaluator config file");
+                    Exceptions.Throw(e, _logger);
+                }
+
+                // remote driver Id
+                string rId = args[0];
+
+                // evaluator configuraiton file
+                string evaluatorConfigurationPath = args[1];
+
+                ICodec<REEFMessage> reefMessageCodec = new REEFMessageCodec();
+
+                _evaluatorConfig = new EvaluatorConfigurations(evaluatorConfigurationPath);
+
+                string rootContextConfigString = _evaluatorConfig.RootContextConfiguration;
+                if (string.IsNullOrWhiteSpace(rootContextConfigString))
+                {
+                    Exceptions.Throw(new ArgumentException("empty or null rootContextConfigString"), _logger);
+                }
+                ContextConfiguration rootContextConfiguration = new ContextConfiguration(rootContextConfigString);
+
+                string taskConfig = _evaluatorConfig.TaskConfiguration;
+                Optional<TaskConfiguration> rootTaskConfig = string.IsNullOrEmpty(taskConfig)
+                                        ? Optional<TaskConfiguration>.Empty()
+                                        : Optional<TaskConfiguration>.Of(
+                                            new TaskConfiguration(taskConfig));
+                string rootServiceConfigString = _evaluatorConfig.RootServiceConfiguration;
+                Optional<ServiceConfiguration> rootServiceConfig = string.IsNullOrEmpty(rootServiceConfigString)
+                                        ? Optional<ServiceConfiguration>.Empty()
+                                        : Optional<ServiceConfiguration>.Of(
+                                            new ServiceConfiguration(
+                                                rootServiceConfigString));
+ 
+                // remoteManager used as client-only in evaluator
+                IRemoteManager<REEFMessage> remoteManager = new DefaultRemoteManager<REEFMessage>(reefMessageCodec);
+                IRemoteIdentifier remoteId = new SocketRemoteIdentifier(NetUtilities.ParseIpEndpoint(rId));
+
+                ConfigurationModule module = new ConfigurationModuleBuilder().Build();
+                IConfiguration clockConfiguraiton = module.Build();
+
+                clock =
+                    TangFactory.GetTang().NewInjector(clockConfiguraiton).GetInstance<RuntimeClock>();
+                    _logger.Log(Level.Info, "Application Id: " + _evaluatorConfig.ApplicationId);
+
+                EvaluatorSettings evaluatorSettings = new EvaluatorSettings(
+                    _evaluatorConfig.ApplicationId,
+                    _evaluatorConfig.EvaluatorId,
+                    _heartbeatPeriodInMs,
+                    _heartbeatMaxRetry,
+                    rootContextConfiguration,
+                    clock,
+                    remoteManager,
+                    _injector);
+
+                HeartBeatManager heartBeatManager = new HeartBeatManager(evaluatorSettings, remoteId);
+                ContextManager contextManager = new ContextManager(heartBeatManager, rootServiceConfig, rootTaskConfig);
+                EvaluatorRuntime evaluatorRuntime = new EvaluatorRuntime(contextManager, heartBeatManager);
+
+                // TODO: repalce with injectionFuture
+                heartBeatManager._evaluatorRuntime = evaluatorRuntime;
+                heartBeatManager._contextManager = contextManager;
+
+                SetRuntimeHanlders(evaluatorRuntime, clock);
+            }
+
+            Task evaluatorTask = Task.Run(new Action(clock.Run));
+            evaluatorTask.Wait();            
+        }
+
+        private static void InitInjector()
+        {
+            string clrRuntimeConfigurationFile = Path.Combine(Directory.GetCurrentDirectory(), "reef", "global",
+                                                                Common.Constants.ClrBridgeRuntimeConfiguration);
+            if (!File.Exists(clrRuntimeConfigurationFile))
+            {
+                var e =
+                    new InvalidOperationException("Cannot find clrRuntimeConfiguration from " +
+                                                    clrRuntimeConfigurationFile);
+                Exceptions.Throw(e, _logger);
+            }
+
+            try
+            {
+                IConfiguration clrBridgeConfiguration =
+                    new AvroConfigurationSerializer().FromFile(clrRuntimeConfigurationFile);
+                _injector = TangFactory.GetTang().NewInjector(clrBridgeConfiguration);
+            }
+            catch (Exception e)
+            {
+                Exceptions.Caught(e, Level.Error, "Cannot obtain injector from clr bridge configuration.", _logger);
+                Exceptions.Throw(
+                    new InvalidOperationException("Cannot obtain injector from clr bridge configuration.", e),
+                    _logger);
+            }
+        }
+
+        private static void SetCustomTraceListners()
+        {
+            ISet<TraceListener> customTraceListeners;
+            CustomTraceListeners listeners = null;
+            try
+            {
+                listeners = _injector.GetInstance<CustomTraceListeners>();
+                customTraceListeners = listeners.Listeners;
+            }
+            catch (Exception e)
+            {
+                Exceptions.Caught(e, Level.Error, _logger);
+                // custom trace listner not set properly, use empty set
+                customTraceListeners = new HashSet<TraceListener>();
+            }
+            foreach (TraceListener listener in customTraceListeners)
+            {
+                Logger.AddTraceListner(listener);
+            }
+            _logger = Logger.GetLogger(typeof(Evaluator));
+            CustomTraceLevel traceLevel = _injector.GetInstance<CustomTraceLevel>();
+            Logger.SetCustomLevel(traceLevel.TraceLevel);
+        }
+
+        private static void UnhandledExceptionHandler(object sender, UnhandledExceptionEventArgs e)
+        {
+            Exception ex = default(Exception);
+            ex = (Exception)e.ExceptionObject;
+            _logger.Log(Level.Error, "Unhandled exception caught in Evaluator.", ex);
+            Exceptions.Throw(new InvalidOperationException("Unhandled exception caught in Evaluator.", ex), _logger);
+        }
+
+        // set the handlers for runtimeclock manually
+        // we only need runtimestart and runtimestop handlers now
+        private static void SetRuntimeHanlders(EvaluatorRuntime evaluatorRuntime, RuntimeClock clock)
+        {
+            HashSet<IObserver<RuntimeStart>> runtimeStarts = new HashSet<IObserver<RuntimeStart>>();
+            runtimeStarts.Add(evaluatorRuntime);
+            InjectionFutureImpl<ISet<IObserver<RuntimeStart>>> injectRuntimeStart = new InjectionFutureImpl<ISet<IObserver<RuntimeStart>>>(runtimeStarts);
+            clock.InjectedRuntimeStartHandler = injectRuntimeStart;
+
+            HashSet<IObserver<RuntimeStop>> runtimeStops = new HashSet<IObserver<RuntimeStop>>();
+            runtimeStops.Add(evaluatorRuntime);
+            InjectionFutureImpl<ISet<IObserver<RuntimeStop>>> injectRuntimeStop = new InjectionFutureImpl<ISet<IObserver<RuntimeStop>>>(runtimeStops);
+            clock.InjectedRuntimeStopHandler = injectRuntimeStop;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-applications/Evaluator/Evaluator.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-applications/Evaluator/Evaluator.csproj b/lang/cs/Source/REEF/reef-applications/Evaluator/Evaluator.csproj
new file mode 100644
index 0000000..dc9c703
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-applications/Evaluator/Evaluator.csproj
@@ -0,0 +1,105 @@
+<?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.
+-->
+<Project ToolsVersion="12.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{1B983182-9C30-464C-948D-F87EB93A8240}</ProjectGuid>
+    <OutputType>Exe</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Org.Apache.Reef.Evaluator</RootNamespace>
+    <AssemblyName>Org.Apache.Reef.Evaluator</AssemblyName>
+    <TargetFrameworkVersion>v4.5</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+    <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..\..\..\</SolutionDir>
+    <RestorePackages>true</RestorePackages>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
+    <PlatformTarget>AnyCPU</PlatformTarget>
+    <DebugSymbols>true</DebugSymbols>
+    <DebugType>full</DebugType>
+    <Optimize>false</Optimize>
+    <OutputPath>..\..\..\..\bin\Debug\Org.Apache.Reef.Evaluator\</OutputPath>
+    <DefineConstants>DEBUG;TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
+    <PlatformTarget>AnyCPU</PlatformTarget>
+    <DebugType>pdbonly</DebugType>
+    <Optimize>true</Optimize>
+    <OutputPath>..\..\..\..\bin\Release\Microsoft.Reef.Evaluator\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="protobuf-net">
+      <HintPath>..\..\..\..\packages\protobuf-net.2.0.0.668\lib\net40\protobuf-net.dll</HintPath>
+    </Reference>
+    <Reference Include="System" />
+    <Reference Include="System.Configuration" />
+    <Reference Include="System.Core" />
+    <Reference Include="System.Xml.Linq" />
+    <Reference Include="System.Data.DataSetExtensions" />
+    <Reference Include="Microsoft.CSharp" />
+    <Reference Include="System.Data" />
+    <Reference Include="System.Xml" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="Evaluator.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="packages.config" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\..\..\Tang\Tang\Tang.csproj">
+      <Project>{97dbb573-3994-417a-9f69-ffa25f00d2a6}</Project>
+      <Name>Tang</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\..\Utilities\Utilities.csproj">
+      <Project>{79e7f89a-1dfb-45e1-8d43-d71a954aeb98}</Project>
+      <Name>Utilities</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\..\WAKE\Wake\Wake.csproj">
+      <Project>{cdfb3464-4041-42b1-9271-83af24cd5008}</Project>
+      <Name>Wake</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\reef-common\ReefCommon\ReefCommon.csproj">
+      <Project>{545a0582-4105-44ce-b99c-b1379514a630}</Project>
+      <Name>ReefCommon</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\reef-common\ReefDriver\ReefDriver.csproj">
+      <Project>{a6baa2a7-f52f-4329-884e-1bcf711d6805}</Project>
+      <Name>ReefDriver</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
+  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
+       Other similar extension points exist, see Microsoft.Common.targets.
+  <Target Name="BeforeBuild">
+  </Target>
+  <Target Name="AfterBuild">
+  </Target>
+  -->
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-applications/Evaluator/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-applications/Evaluator/Properties/AssemblyInfo.cs b/lang/cs/Source/REEF/reef-applications/Evaluator/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..51ff356
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-applications/Evaluator/Properties/AssemblyInfo.cs
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System.Reflection;
+using System.Runtime.CompilerServices;
+using System.Runtime.InteropServices;
+
+// General Information about an assembly is controlled through the following 
+// set of attributes. Change these attribute values to modify the information
+// associated with an assembly.
+[assembly: AssemblyTitle("Evaluator")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("Evaluator")]
+[assembly: AssemblyCopyright("Copyright ©  2015")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+// Setting ComVisible to false makes the types in this assembly not visible 
+// to COM components.  If you need to access a type in this assembly from 
+// COM, set the ComVisible attribute to true on that type.
+[assembly: ComVisible(false)]
+
+// The following GUID is for the ID of the typelib if this project is exposed to COM
+[assembly: Guid("a64dc535-9b1e-41a4-8303-117f8b28c8c0")]
+
+// Version information for an assembly consists of the following four values:
+//
+//      Major Version
+//      Minor Version 
+//      Build Number
+//      Revision
+//
+// You can specify all the values or you can default the Build and Revision Numbers 
+// by using the '*' as shown below:
+// [assembly: AssemblyVersion("1.0.*")]
+[assembly: AssemblyVersion("1.0.0.0")]
+[assembly: AssemblyFileVersion("1.0.0.0")]

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-applications/Evaluator/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-applications/Evaluator/packages.config b/lang/cs/Source/REEF/reef-applications/Evaluator/packages.config
new file mode 100644
index 0000000..81b0ef5
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-applications/Evaluator/packages.config
@@ -0,0 +1,22 @@
+<?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="protobuf-net" version="2.0.0.668" 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/ClientJobStatusHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/ClientJobStatusHandler.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/ClientJobStatusHandler.cs
new file mode 100644
index 0000000..508d3b3
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/ClientJobStatusHandler.cs
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Client;
+using Org.Apache.Reef.Common.ProtoBuf.ClienRuntimeProto;
+using Org.Apache.Reef.Common.ProtoBuf.ReefProtocol;
+using Org.Apache.Reef.Common.ProtoBuf.ReefServiceProto;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Wake.Remote;
+using Org.Apache.Reef.Wake.Time;
+using System;
+
+namespace Org.Apache.Reef.Common
+{
+    public class ClientJobStatusHandler : IJobMessageObserver, IObserver<StartTime>
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(ClientJobStatusHandler));
+
+        private IClock _clock;
+
+        private string _jobId;
+
+        private IObserver<JobStatusProto> _jobStatusHandler;
+
+        private IDisposable _jobControlChannel;
+
+        State _state = State.INIT;
+
+        public ClientJobStatusHandler(
+            IRemoteManager<IRemoteMessage<REEFMessage>> remoteManager,
+            IClock clock,
+            IObserver<JobControlProto> jobControlHandler,
+            string jobId,
+            string clientRID)
+        {
+            _clock = clock;
+            _jobId = jobId;
+            _jobStatusHandler = null;
+            _jobControlChannel = null;
+            //_jobStatusHandler = remoteManager.GetRemoteObserver()
+            //_jobControlChannel = remoteManager.RegisterObserver()
+        }
+
+        public void Dispose(Optional<Exception> e)
+        {
+            try
+            {
+                if (e.IsPresent())
+                {
+                    OnError(e.Value);
+                }
+                else
+                {
+                    JobStatusProto proto = new JobStatusProto();
+                    proto.identifier = _jobId;
+                    proto.state = State.DONE;
+                    Send(proto);
+                }
+            }
+            catch (Exception ex)
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.CaughtAndThrow(ex, Level.Warning, "Error closing ClientJobStatusHandler", LOGGER);
+            }
+
+            try
+            {
+                _jobControlChannel.Dispose();
+            }
+            catch (Exception ex)
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.CaughtAndThrow(ex, Level.Warning, "Error closing jobControlChannel", LOGGER);
+            }
+        }
+
+        public void OnNext(byte[] value)
+        {
+            LOGGER.Log(Level.Info, "Job message from {0}" + _jobId);   
+            SendInit();
+            JobStatusProto proto = new JobStatusProto();
+            proto.identifier = _jobId;
+            proto.state = State.RUNNING;
+            proto.message = value;
+            Send(proto);
+        }
+
+        public void OnNext(StartTime value)
+        {
+            LOGGER.Log(Level.Info, "StartTime:" + value);
+            SendInit();
+        }
+
+        public void OnError(Exception error)
+        {
+            LOGGER.Log(Level.Error, "job excemption", error);
+            JobStatusProto proto = new JobStatusProto();
+            proto.identifier = _jobId;
+            proto.state = State.FAILED;
+            proto.exception = ByteUtilities.StringToByteArrays(error.Message);
+            _clock.Dispose();
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+
+        private void Send(JobStatusProto status)
+        {
+            LOGGER.Log(Level.Info, "Sending job status " + status);
+            _jobStatusHandler.OnNext(status);
+        }
+
+        private void SendInit()
+        {
+            if (_state == State.INIT)
+            {
+                JobStatusProto proto = new JobStatusProto();
+                proto.identifier = _jobId;
+                proto.state = State.INIT;
+                Send(proto);
+                _state = State.RUNNING;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/Constants.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/Constants.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/Constants.cs
new file mode 100644
index 0000000..ea9ce76
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/Constants.cs
@@ -0,0 +1,38 @@
+/**
+ * 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
+{
+    public class Constants
+    {
+        public const string ClrBridgeRuntimeConfiguration = "clrBridge.config";
+
+        // if 8080 port is not used, then query would fail, 
+        // this is only for local runtime testing purpose though, so it should be ok
+        public const string LocalHttpEndpointBaseUri = @"http://localhost:8080/";  
+
+        public const string HDInsightClusterHttpEndpointBaseUri = @"http://headnodehost:9014/proxy/";
+
+        public const string HttpReefUriSpecification = @"Reef/v1/";
+
+        public const string HttpDriverUriTarget = @"Driver/";
+
+        public const string NameServerServiceName = "NameServer";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/EvaluatorHeartBeatSanityChecker.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/EvaluatorHeartBeatSanityChecker.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/EvaluatorHeartBeatSanityChecker.cs
new file mode 100644
index 0000000..38ed6c0
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/EvaluatorHeartBeatSanityChecker.cs
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Utilities.Logging;
+using System;
+using System.Collections.Generic;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Common
+{
+    public class EvaluatorHeartBeatSanityChecker
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(EvaluatorHeartBeatSanityChecker));
+        
+        Dictionary<string, long> _timeStamps = new Dictionary<string, long>();
+
+        public void check(string id, long timeStamp)
+        {
+            lock (this)
+            {
+                if (_timeStamps.ContainsKey(id))
+                {
+                    long oldTimeStamp = _timeStamps[id];
+                    LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "TIMESTAMP CHECKER: id [{0}], old timestamp [{1}], new timestamp [{2}]", id, oldTimeStamp, timeStamp));
+                    if (oldTimeStamp > timeStamp)
+                    {
+                        string msg = string.Format(
+                            CultureInfo.InvariantCulture,
+                            "Received an old heartbeat with timestamp [{0}] while timestamp [{1}] was received earlier",
+                            oldTimeStamp,
+                            timeStamp);
+                        Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new InvalidOperationException(msg), LOGGER);
+                    }
+                }
+                _timeStamps.Add(id, timeStamp);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/FailedRuntime.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/FailedRuntime.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/FailedRuntime.cs
new file mode 100644
index 0000000..c700986
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/FailedRuntime.cs
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Common.Api;
+using Org.Apache.Reef.Common.ProtoBuf.ReefServiceProto;
+using Org.Apache.Reef.Utilities;
+using System;
+
+namespace Org.Apache.Reef.Common
+{
+    public class FailedRuntime : AbstractFailure
+    {
+        public FailedRuntime(RuntimeErrorProto error)
+            : base(error.identifier, error.message, null, GetException(error), error.exception)
+        {
+        }
+
+        /// <summary>
+        /// Get the exception from error
+        /// </summary>
+        /// <param name="error"></param>
+        /// <returns>excetpiont from error</returns>
+        private static Exception GetException(RuntimeErrorProto error)
+        {
+            byte[] data = error.exception;
+            if (data != null)
+            {
+                return new InvalidOperationException(ByteUtilities.ByteArrarysToString(error.exception));
+            }
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/IContextAndTaskSubmittable.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/IContextAndTaskSubmittable.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/IContextAndTaskSubmittable.cs
new file mode 100644
index 0000000..edf983f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/IContextAndTaskSubmittable.cs
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Tang.Interface;
+
+namespace Org.Apache.Reef.Common
+{
+    /// <summary>
+    /// Base interface for classes that support the simultaneous submission of both Context and Task configurations.
+    /// </summary>
+    public interface IContextAndTaskSubmittable
+    {
+        /// <summary>
+        /// Submit a Context and an Task.
+        /// The semantics of this call are the same as first submitting the context and then, on the fired ActiveContext event
+        /// to submit the Task. The performance of this will be better, though as it potentially saves some roundtrips on
+        /// the network.
+        /// REEF will not fire an ActiveContext as a result of this. Instead, it will fire a TaskRunning event.
+        /// </summary>
+        /// <param name="contextConfiguration"> the Configuration of the EvaluatorContext. See ContextConfiguration for details.</param>
+        /// <param name="taskConfiguration">the Configuration of the Task. See TaskConfiguration for details.</param>
+        void SubmitContextAndTask(IConfiguration contextConfiguration, IConfiguration taskConfiguration);
+
+        /// <summary>
+        /// Submit a Context with Services and an Task.
+        /// The semantics of this call are the same as first submitting the context and services and then, on the fired
+        /// ActiveContext event to submit the Task. The performance of this will be better, though as it potentially saves
+        /// some roundtrips on the network.
+        /// REEF will not fire an ActiveContext as a result of this. Instead, it will fire a TaskRunning event.
+        /// </summary>
+        /// <param name="contextConfiguration"></param>
+        /// <param name="serviceConfiguration"></param>
+        /// <param name="taskConfiguration"></param>
+        void SubmitContextAndServiceAndTask(
+            IConfiguration contextConfiguration, 
+            IConfiguration serviceConfiguration, 
+            IConfiguration taskConfiguration);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/IContextSubmittable.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/IContextSubmittable.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/IContextSubmittable.cs
new file mode 100644
index 0000000..039d2a3
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/IContextSubmittable.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.Tang.Interface;
+
+namespace Org.Apache.Reef.Common
+{
+    /// <summary>
+    ///  Base interface for classes that support Context submission.
+    /// </summary>
+    public interface IContextSubmittable
+    {
+        /// <summary>
+        ///  Submit a Context.
+        /// </summary>
+        /// <param name="contextConfiguration">the Configuration of the EvaluatorContext. See ContextConfiguration for details.</param>
+        void SubmitContext(IConfiguration contextConfiguration);
+
+        /// <summary>
+        /// Submit a Context and a Service Configuration.
+        /// </summary>
+        /// <param name="contextConfiguration">the Configuration of the EvaluatorContext. See ContextConfiguration for details.</param>
+        /// <param name="serviceConfiguration">the Configuration for the Services. See ServiceConfiguration for details.</param>
+        void SubmitContextAndService(IConfiguration contextConfiguration, IConfiguration serviceConfiguration);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/IJobMessageObserver.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/IJobMessageObserver.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/IJobMessageObserver.cs
new file mode 100644
index 0000000..a5be5d5
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/IJobMessageObserver.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;
+
+namespace Org.Apache.Reef.Common.Client
+{
+    /// <summary>
+    ///  The driver uses this interface to communicate with the job client.
+    /// </summary>
+    public interface IJobMessageObserver : IObserver<byte[]>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/ITaskSubmittable.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/ITaskSubmittable.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/ITaskSubmittable.cs
new file mode 100644
index 0000000..1cc9312
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/ITaskSubmittable.cs
@@ -0,0 +1,35 @@
+/**
+ * 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.Interface;
+
+namespace Org.Apache.Reef.Common
+{
+    /// <summary>
+    ///  Base interface for classes that support Task submission.
+    /// </summary>
+    public interface ITaskSubmittable
+    {
+       /// <summary>
+        /// Submits an Task (encoded in the Configuration) for execution.
+       /// </summary>
+        /// <param name="taskConf">the Configuration. See TaskConfiguration for details</param>
+        void SubmitTask(IConfiguration taskConf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/Properties/AssemblyInfo.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..f6c13bd
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/Properties/AssemblyInfo.cs
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System.Reflection;
+using System.Runtime.CompilerServices;
+using System.Runtime.InteropServices;
+
+// General Information about an assembly is controlled through the following 
+// set of attributes. Change these attribute values to modify the information
+// associated with an assembly.
+[assembly: AssemblyTitle("ReefCommon")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("ReefCommon")]
+[assembly: AssemblyCopyright("Copyright ©  2015")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+// Setting ComVisible to false makes the types in this assembly not visible 
+// to COM components.  If you need to access a type in this assembly from 
+// COM, set the ComVisible attribute to true on that type.
+[assembly: ComVisible(false)]
+
+// The following GUID is for the ID of the typelib if this project is exposed to COM
+[assembly: Guid("a810ee4a-fe13-4536-9e9c-5275b16e0842")]
+
+// Version information for an assembly consists of the following four values:
+//
+//      Major Version
+//      Minor Version 
+//      Build Number
+//      Revision
+//
+// You can specify all the values or you can default the Build and Revision Numbers 
+// by using the '*' as shown below:
+// [assembly: AssemblyVersion("1.0.*")]
+[assembly: AssemblyVersion("1.0.0.0")]
+[assembly: AssemblyFileVersion("1.0.0.0")]

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/ReefCommon.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/ReefCommon.csproj b/lang/cs/Source/REEF/reef-common/ReefCommon/ReefCommon.csproj
new file mode 100644
index 0000000..6dc5b4b
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/ReefCommon.csproj
@@ -0,0 +1,215 @@
+<?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.
+-->
+<Project ToolsVersion="12.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{545A0582-4105-44CE-B99C-B1379514A630}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Org.Apache.Reef.Common</RootNamespace>
+    <AssemblyName>Org.Apache.Reef.Common</AssemblyName>
+    <TargetFrameworkVersion>v4.5</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+    <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..\..\..\</SolutionDir>
+    <RestorePackages>true</RestorePackages>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
+    <PlatformTarget>AnyCPU</PlatformTarget>
+    <DebugSymbols>true</DebugSymbols>
+    <DebugType>full</DebugType>
+    <Optimize>false</Optimize>
+    <OutputPath>..\..\..\..\bin\Debug\Org.Apache.Reef.Common\</OutputPath>
+    <DefineConstants>DEBUG;TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
+    <PlatformTarget>AnyCPU</PlatformTarget>
+    <DebugType>pdbonly</DebugType>
+    <Optimize>true</Optimize>
+    <OutputPath>..\..\..\..\bin\Release\Microsoft.Reef.Common\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="Microsoft.Hadoop.Avro">
+      <HintPath>..\..\..\..\packages\Microsoft.Hadoop.Avro.1.4.0.0\lib\net40\Microsoft.Hadoop.Avro.dll</HintPath>
+    </Reference>
+    <Reference Include="Newtonsoft.Json">
+      <HintPath>..\..\..\..\packages\Newtonsoft.Json.6.0.8\lib\net45\Newtonsoft.Json.dll</HintPath>
+    </Reference>
+    <Reference Include="protobuf-net">
+      <HintPath>..\..\..\..\packages\protobuf-net.2.0.0.668\lib\net40\protobuf-net.dll</HintPath>
+    </Reference>
+    <Reference Include="System" />
+    <Reference Include="System.Core" />
+    <Reference Include="System.Reactive.Core">
+      <HintPath>..\..\..\..\packages\Rx-Core.2.2.5\lib\net45\System.Reactive.Core.dll</HintPath>
+    </Reference>
+    <Reference Include="System.Reactive.Interfaces">
+      <HintPath>..\..\..\..\packages\Rx-Interfaces.2.2.5\lib\net45\System.Reactive.Interfaces.dll</HintPath>
+    </Reference>
+    <Reference Include="System.Runtime.Serialization" />
+    <Reference Include="System.Xml.Linq" />
+    <Reference Include="System.Data.DataSetExtensions" />
+    <Reference Include="Microsoft.CSharp" />
+    <Reference Include="System.Data" />
+    <Reference Include="System.Xml" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="api\AbstractFailure.cs" />
+    <Compile Include="api\IAbstractFailure.cs" />
+    <Compile Include="api\IFailure.cs" />
+    <Compile Include="api\IResourceLaunchHandler.cs" />
+    <Compile Include="api\IResourceReleaseHandler.cs" />
+    <Compile Include="api\IResourceRequestHandler.cs" />
+    <Compile Include="avro\AvroDriverInfo.cs" />
+    <Compile Include="avro\AvroHttpRequest.cs" />
+    <Compile Include="avro\AvroHttpSerializer.cs" />
+    <Compile Include="avro\AvroJsonSerializer.cs" />
+    <Compile Include="avro\AvroReefServiceInfo.cs" />
+    <Compile Include="catalog\capabilities\CPU.cs" />
+    <Compile Include="catalog\capabilities\ICapability.cs" />
+    <Compile Include="catalog\capabilities\RAM.cs" />
+    <Compile Include="catalog\INodeDescriptor.cs" />
+    <Compile Include="catalog\IRackDescriptor.cs" />
+    <Compile Include="catalog\IResourceCatalog.cs" />
+    <Compile Include="catalog\NodeDescriptorImpl.cs" />
+    <Compile Include="catalog\RackDescriptorImpl.cs" />
+    <Compile Include="catalog\ResourceCatalogImpl.cs" />
+    <Compile Include="ClientJobStatusHandler.cs" />
+    <Compile Include="Constants.cs" />
+    <Compile Include="context\ContextMessage.cs" />
+    <Compile Include="context\IContextMessage.cs" />
+    <Compile Include="context\IContextMessageHandler.cs" />
+    <Compile Include="context\IContextMessageSource.cs" />
+    <Compile Include="EvaluatorHeartBeatSanityChecker.cs" />
+    <Compile Include="evaluator\DefaultLocalHttpDriverConnection.cs" />
+    <Compile Include="evaluator\DefaultYarnClusterHttpDriverConnection.cs" />
+    <Compile Include="evaluator\DefaultYarnOneBoxHttpDriverConnection.cs" />
+    <Compile Include="evaluator\DriverInformation.cs" />
+    <Compile Include="evaluator\EvaluatorOperationState.cs" />
+    <Compile Include="evaluator\EvaluatorRuntimeState.cs" />
+    <Compile Include="evaluator\EvaluatorType.cs" />
+    <Compile Include="evaluator\IDriverConnection.cs" />
+    <Compile Include="events\IContextStart.cs" />
+    <Compile Include="events\IContextStop.cs" />
+    <Compile Include="exceptions\EvaluatorException.cs" />
+    <Compile Include="exceptions\JobException.cs" />
+    <Compile Include="FailedRuntime.cs" />
+    <Compile Include="IContextAndTaskSubmittable.cs" />
+    <Compile Include="IContextSubmittable.cs" />
+    <Compile Include="IJobMessageObserver.cs" />
+    <Compile Include="io\INameClient.cs" />
+    <Compile Include="io\NameAssignment.cs" />
+    <Compile Include="io\NamingConfiguration.cs" />
+    <Compile Include="io\NamingConfigurationOptions.cs" />
+    <Compile Include="ITaskSubmittable.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="protobuf\cs\ClientRuntime.pb.cs" />
+    <Compile Include="protobuf\cs\codec\EvaluatorHeartbeatProtoCodec.cs" />
+    <Compile Include="protobuf\cs\codec\REEFMessageCodec.cs" />
+    <Compile Include="protobuf\cs\DriverRuntime.pb.cs" />
+    <Compile Include="protobuf\cs\EvaluatorRunTime.pb.cs" />
+    <Compile Include="protobuf\cs\ReefProtocol.pb.cs" />
+    <Compile Include="protobuf\cs\ReefService.pb.cs" />
+    <Compile Include="protobuf\cs\Serializer.cs" />
+    <Compile Include="runtime\evaluator\Constants.cs" />
+    <Compile Include="runtime\evaluator\context\ContextClientCodeException.cs" />
+    <Compile Include="runtime\evaluator\context\ContextConfiguration.cs" />
+    <Compile Include="runtime\evaluator\context\ContextLifeCycle.cs" />
+    <Compile Include="runtime\evaluator\context\ContextManager.cs" />
+    <Compile Include="runtime\evaluator\context\ContextRuntime.cs" />
+    <Compile Include="runtime\evaluator\context\ContextStartImpl.cs" />
+    <Compile Include="runtime\evaluator\context\ContextStopImpl.cs" />
+    <Compile Include="runtime\evaluator\context\RootContextLauncher.cs" />
+    <Compile Include="runtime\evaluator\EvaluatorRuntime.cs" />
+    <Compile Include="runtime\evaluator\EvaluatorSettings.cs" />
+    <Compile Include="runtime\evaluator\HeartBeatManager.cs" />
+    <Compile Include="runtime\evaluator\ReefMessageProtoObserver.cs" />
+    <Compile Include="runtime\evaluator\task\CloseEventImpl.cs" />
+    <Compile Include="runtime\evaluator\task\DriverMessageImpl.cs" />
+    <Compile Include="runtime\evaluator\task\SuspendEventImpl.cs" />
+    <Compile Include="runtime\evaluator\task\TaskClientCodeException.cs" />
+    <Compile Include="runtime\evaluator\task\TaskLifeCycle.cs" />
+    <Compile Include="runtime\evaluator\task\TaskRuntime.cs" />
+    <Compile Include="runtime\evaluator\task\TaskStartImpl.cs" />
+    <Compile Include="runtime\evaluator\task\TaskState.cs" />
+    <Compile Include="runtime\evaluator\task\TaskStatus.cs" />
+    <Compile Include="runtime\evaluator\task\TaskStopImpl.cs" />
+    <Compile Include="runtime\evaluator\utils\EvaluatorConfigurations.cs" />
+    <Compile Include="runtime\evaluator\utils\RemoteManager.cs" />
+    <Compile Include="runtime\MachineStatus.cs" />
+    <Compile Include="services\IService.cs" />
+    <Compile Include="services\ServiceConfiguration.cs" />
+    <Compile Include="services\ServicesConfigurationOptions.cs" />
+    <Compile Include="tasks\defaults\DefaultDriverMessageHandler.cs" />
+    <Compile Include="tasks\defaults\DefaultTaskMessageSource.cs" />
+    <Compile Include="tasks\events\ICloseEvent.cs" />
+    <Compile Include="tasks\events\IDriverMessage.cs" />
+    <Compile Include="tasks\events\ISuspendEvent.cs" />
+    <Compile Include="tasks\events\ITaskStart.cs" />
+    <Compile Include="tasks\events\ITaskStop.cs" />
+    <Compile Include="tasks\IDriverMessageHandler.cs" />
+    <Compile Include="tasks\IRunningTask.cs" />
+    <Compile Include="tasks\ITask.cs" />
+    <Compile Include="tasks\ITaskMessageSource.cs" />
+    <Compile Include="tasks\TaskConfiguration.cs" />
+    <Compile Include="tasks\TaskConfigurationOptions.cs" />
+    <Compile Include="tasks\TaskMessage.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="packages.config" />
+    <None Include="protobuf\proto\client_runtime.proto" />
+    <None Include="protobuf\proto\driver_runtime.proto" />
+    <None Include="protobuf\proto\evaluator_runtime.proto" />
+    <None Include="protobuf\proto\reef_protocol.proto" />
+    <None Include="protobuf\proto\reef_service_protos.proto" />
+  </ItemGroup>
+  <ItemGroup>
+    <Folder Include="protobuf\tools\" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\..\..\Tang\Tang\Tang.csproj">
+      <Project>{97dbb573-3994-417a-9f69-ffa25f00d2a6}</Project>
+      <Name>Tang</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\..\Utilities\Utilities.csproj">
+      <Project>{79e7f89a-1dfb-45e1-8d43-d71a954aeb98}</Project>
+      <Name>Utilities</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\..\WAKE\Wake\Wake.csproj">
+      <Project>{cdfb3464-4041-42b1-9271-83af24cd5008}</Project>
+      <Name>Wake</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
+  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
+       Other similar extension points exist, see Microsoft.Common.targets.
+  <Target Name="BeforeBuild">
+  </Target>
+  <Target Name="AfterBuild">
+  </Target>
+  -->
+</Project>
\ 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/api/AbstractFailure.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/api/AbstractFailure.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/api/AbstractFailure.cs
new file mode 100644
index 0000000..4b9c1a6
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/api/AbstractFailure.cs
@@ -0,0 +1,142 @@
+/**
+ * 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;
+using System.Globalization;
+using Org.Apache.Reef.Utilities.Logging;
+
+namespace Org.Apache.Reef.Common.Api
+{
+    public abstract class AbstractFailure : IFailure
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(AbstractFailure));
+
+        public AbstractFailure()
+        {
+        }
+
+        /// <summary>
+        /// Most detailed error message constructor that takes all parameters possible.
+        /// </summary>
+        /// <param name="id">Identifier of the entity that produced the error. Cannot be null.</param>
+        /// <param name="message">One-line error message. Cannot be null.</param>
+        /// <param name="description">Long error description. Can be null.</param>
+        /// <param name="cause">Exception that caused the error. Can be null.</param>
+        /// <param name="data">byte array that contains serialized version of the error. Can be null.</param>
+        public AbstractFailure(string id, string message, string description, Exception cause, byte[] data)
+        {
+            if (string.IsNullOrEmpty(id))
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new ArgumentException("id"), LOGGER);
+            }
+            if (string.IsNullOrEmpty(message))
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new ArgumentException("message"), LOGGER);
+            }
+            Id = id;
+            Message = message;
+            Description = Optional<string>.OfNullable(string.IsNullOrEmpty(description) ? GetStackTrace(cause) : description);
+            Cause = Optional<Exception>.OfNullable(cause);
+            Data = Optional<byte[]>.OfNullable(data);
+        }
+
+        /// <summary>
+        ///  Build error message given the entity ID and the short error message.
+        /// </summary>
+        /// <param name="id"></param>
+        /// <param name="message"></param>
+        public AbstractFailure(string id, string message)
+            : this(id, message, null, null, null)
+        {
+        }
+
+        /// <summary>
+        ///  Build error message given the failed entity ID and  Exception.
+        ///  Populates the message with the Exception.getMessage() result, and stores
+        ///  the exception stack trace in the description.
+        /// </summary>
+        /// <param name="id"></param>
+        /// <param name="cause"></param>
+        public AbstractFailure(string id, Exception cause)
+        {
+            if (string.IsNullOrEmpty(id))
+            {
+                Org.Apache.Reef.Utilities.Diagnostics.Exceptions.Throw(new ArgumentException("id"), LOGGER);
+            }
+            Id = id;
+            Message = cause.Message;
+            Description = Optional<string>.Of(GetStackTrace(cause));
+            Cause = Optional<Exception>.Of(cause);
+            Data = Optional<byte[]>.Empty();
+        }
+
+        /// <summary>
+        /// Build error message given the entity ID plus short and long error message.
+        /// </summary>
+        /// <param name="id"></param>
+        /// <param name="message"></param>
+        /// <param name="description"></param>
+        public AbstractFailure(string id, string message, string description)
+            : this(id, message, description, null, null)
+        {
+        }
+
+        /// <summary>
+        /// Identifier of the entity that produced the error. Cannot be null.
+        /// </summary>
+        public string Id { get; set; }
+
+        public string Message { get; set; }
+
+        public Optional<string> Description { get; set; }
+
+        public Optional<string> Reason { get; set; }
+
+        public Optional<Exception> Cause { get; set; }
+
+        public Optional<byte[]> Data { get; set; }
+
+        public Exception AsError()
+        {
+            return Cause.IsPresent() ? Cause.Value : new InvalidOperationException(ToString());
+        }
+
+        /// <summary>
+        ///  Helper function: produce the string that contains the given exception's stack trace. Returns null if the argument is null.
+        /// </summary>
+        /// <param name="ex"></param>
+        public string GetStackTrace(Exception ex)
+        {
+            if (ex == null)
+            {
+                return null;
+            }
+            else
+            {
+                return ex.StackTrace;
+            }
+        }
+
+        public override string ToString()
+        {
+            return string.Format(CultureInfo.InvariantCulture, "{0} with id={1} failed: {2}", GetType(), Id, Message);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/api/IAbstractFailure.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/api/IAbstractFailure.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IAbstractFailure.cs
new file mode 100644
index 0000000..410eacb
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IAbstractFailure.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.Api
+{
+    public interface IAbstractFailure : IFailure
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/api/IFailure.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/api/IFailure.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IFailure.cs
new file mode 100644
index 0000000..02382d0
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IFailure.cs
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Utilities;
+using System;
+
+namespace Org.Apache.Reef.Common.Api
+{
+    /// <summary>
+    /// Common interface for all error messages in REEF.
+    /// Most of its functionality is generic and implemented in the AbstractFailure class.
+    /// </summary>
+    public interface IFailure : IIdentifiable
+    {
+        /// <summary>
+        /// One-line error message. Should never be null.
+        /// </summary>
+        string Message { get; set; }
+
+        /// <summary>
+        ///  Optional long error description.
+        /// </summary>
+        Optional<string> Description { get; set; }
+
+        /// <summary>
+        /// Exception that caused the error, or null.
+        /// </summary>
+        Optional<string> Reason { get; set; }
+
+        /// <summary>
+        /// Optional serialized version of the error message.
+        /// </summary>
+        Optional<byte[]> Data { get; set; }
+
+        /// <summary>
+        /// Return the original Java Exception, or generate a new one if it does not exists.
+        /// ALWAYS returns an exception, never null
+        /// </summary>
+        Exception AsError();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceLaunchHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceLaunchHandler.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceLaunchHandler.cs
new file mode 100644
index 0000000..edea908
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceLaunchHandler.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.Common.ProtoBuf.DriverRuntimeProto;
+using System;
+
+namespace Org.Apache.Reef.Common.Api
+{
+    public interface IResourceLaunchHandler : IObserver<ResourceLaunchProto>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceReleaseHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceReleaseHandler.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceReleaseHandler.cs
new file mode 100644
index 0000000..bcc93ba
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceReleaseHandler.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.Common.ProtoBuf.DriverRuntimeProto;
+using System;
+
+namespace Org.Apache.Reef.Common.Api
+{
+    public interface IResourceReleaseHandler : IObserver<ResourceReleaseProto>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceRequestHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceRequestHandler.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceRequestHandler.cs
new file mode 100644
index 0000000..9eda5c8
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/api/IResourceRequestHandler.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.Common.ProtoBuf.DriverRuntimeProto;
+using System;
+
+namespace Org.Apache.Reef.Common.Api
+{
+    public interface IResourceRequestHandler : IObserver<ResourceRequestProto>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroDriverInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroDriverInfo.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroDriverInfo.cs
new file mode 100644
index 0000000..7a54f4f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroDriverInfo.cs
@@ -0,0 +1,65 @@
+//<auto-generated />
+namespace Org.Apache.Reef.Common.Avro
+{
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.webserver.AvroDriverInfo.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.webserver")]
+    [KnownType(typeof(List<Org.Apache.Reef.Common.Avro.AvroReefServiceInfo>))]
+    public partial class AvroDriverInfo
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.webserver.AvroDriverInfo"",""fields"":[{""name"":""remoteId"",""type"":""string""},{""name"":""startTime"",""type"":""string""},{""name"":""services"",""type"":{""type"":""array"",""items"":{""type"":""record"",""name"":""org.apache.reef.webserver.AvroReefServiceInfo"",""fields"":[{""name"":""serviceName"",""type"":""string""},{""name"":""serviceInfo"",""type"":""string""}]}}}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the remoteId field.
+        /// </summary>
+        [DataMember]
+        public string remoteId { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the startTime field.
+        /// </summary>
+        [DataMember]
+        public string startTime { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the services field.
+        /// </summary>
+        [DataMember]
+        public IList<Org.Apache.Reef.Common.Avro.AvroReefServiceInfo> services { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroDriverInfo"/> class.
+        /// </summary>
+        public AvroDriverInfo()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroDriverInfo"/> class.
+        /// </summary>
+        /// <param name="remoteId">The remoteId.</param>
+        /// <param name="startTime">The startTime.</param>
+        /// <param name="services">The services.</param>
+        public AvroDriverInfo(string remoteId, string startTime, IList<Org.Apache.Reef.Common.Avro.AvroReefServiceInfo> services)
+        {
+            this.remoteId = remoteId;
+            this.startTime = startTime;
+            this.services = services;
+        }
+    }
+}
\ 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/avro/AvroHttpRequest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroHttpRequest.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroHttpRequest.cs
new file mode 100644
index 0000000..1e22569
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroHttpRequest.cs
@@ -0,0 +1,79 @@
+//<auto-generated />
+namespace Org.Apache.Reef.Common.Avro
+{
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.webserver.AvroHttpRequest.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.webserver")]
+    public partial class AvroHttpRequest
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.webserver.AvroHttpRequest"",""fields"":[{""name"":""requestUrl"",""type"":""string""},{""name"":""pathInfo"",""type"":""string""},{""name"":""queryString"",""type"":""string""},{""name"":""httpMethod"",""type"":""string""},{""name"":""inputStream"",""type"":""bytes""}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+
+        /// <summary>
+        /// Gets or sets the requestUrl field.
+        /// </summary>
+        [DataMember]
+        public string RequestUrl { get; set; }
+
+        /// <summary>
+        /// Gets or sets the pathInfo field.
+        /// </summary>
+        [DataMember]
+        public string PathInfo { get; set; }
+
+        /// <summary>
+        /// Gets or sets the queryString field.
+        /// </summary>
+        [DataMember]
+        public string QueryString { get; set; }
+
+        /// <summary>
+        /// Gets or sets the httpMethod field.
+        /// </summary>
+        [DataMember]
+        public string HttpMethod { get; set; }
+
+        /// <summary>
+        /// Gets or sets the inputStream field.
+        /// </summary>
+        [DataMember]
+        public byte[] InputStream { get; set; }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroHttpRequest"/> class.
+        /// </summary>
+        public AvroHttpRequest()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroHttpRequest"/> class.
+        /// </summary>
+        /// <param name="requestUrl">The requestUrl.</param>
+        /// <param name="pathInfo">The pathInfo.</param>
+        /// <param name="queryString">The queryString.</param>
+        /// <param name="httpMethod">The httpMethod.</param>
+        /// <param name="inputStream">The inputStream.</param>
+        public AvroHttpRequest(string requestUrl, string pathInfo, string queryString, string httpMethod, byte[] inputStream)
+        {
+            RequestUrl = requestUrl;
+            PathInfo = pathInfo;
+            QueryString = queryString;
+            HttpMethod = httpMethod;
+            InputStream = inputStream;
+        }
+    }
+}
\ 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/avro/AvroHttpSerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroHttpSerializer.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroHttpSerializer.cs
new file mode 100644
index 0000000..886658f
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroHttpSerializer.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 Microsoft.Hadoop.Avro;
+using System.IO;
+
+namespace Org.Apache.Reef.Common.Avro
+{
+    public class AvroHttpSerializer
+    {
+        public static AvroHttpRequest FromBytes(byte[] serializedBytes)
+        {
+            var serializer = AvroSerializer.Create<AvroHttpRequest>();
+            using (var stream = new MemoryStream(serializedBytes))
+            {
+                return serializer.Deserialize(stream);
+            }
+        }
+    }
+}
\ 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/avro/AvroJsonSerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroJsonSerializer.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroJsonSerializer.cs
new file mode 100644
index 0000000..9158a16
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroJsonSerializer.cs
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Utilities;
+
+using Newtonsoft.Json;
+
+namespace Org.Apache.Reef.Common.Avro
+{
+    /// <summary>
+    /// Wrapper class for serialize/deserialize Avro json. This avoids having to reference Avro dll in every project 
+    /// </summary>
+    /// <typeparam name="T"> the deserialized type</typeparam>
+    public class AvroJsonSerializer<T>
+    {
+        public static T FromString(string str)
+        {
+            return JsonConvert.DeserializeObject<T>(str);
+        }
+
+        public static string ToString(T obj)
+        {
+            return JsonConvert.SerializeObject(obj);
+        }
+
+        public static T FromBytes(byte[] bytes)
+        {
+            return FromString(ByteUtilities.ByteArrarysToString(bytes));
+        }
+
+        public static byte[] ToBytes(T obj)
+        {
+            return ByteUtilities.StringToByteArrays(JsonConvert.SerializeObject(obj));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroReefServiceInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroReefServiceInfo.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroReefServiceInfo.cs
new file mode 100644
index 0000000..671b067
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/avro/AvroReefServiceInfo.cs
@@ -0,0 +1,55 @@
+//<auto-generated />
+namespace Org.Apache.Reef.Common.Avro
+{
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.webserver.AvroReefServiceInfo.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.webserver")]
+    public partial class AvroReefServiceInfo
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.webserver.AvroReefServiceInfo"",""fields"":[{""name"":""serviceName"",""type"":""string""},{""name"":""serviceInfo"",""type"":""string""}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the serviceName field.
+        /// </summary>
+        [DataMember]
+        public string serviceName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the serviceInfo field.
+        /// </summary>
+        [DataMember]
+        public string serviceInfo { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroReefServiceInfo"/> class.
+        /// </summary>
+        public AvroReefServiceInfo()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroReefServiceInfo"/> class.
+        /// </summary>
+        /// <param name="serviceName">The serviceName.</param>
+        /// <param name="serviceInfo">The serviceInfo.</param>
+        public AvroReefServiceInfo(string serviceName, string serviceInfo)
+        {
+            this.serviceName = serviceName;
+            this.serviceInfo = serviceInfo;
+        }
+    }
+}
\ 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/catalog/INodeDescriptor.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/INodeDescriptor.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/INodeDescriptor.cs
new file mode 100644
index 0000000..672a0cd
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/INodeDescriptor.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 System.Net;
+
+using Org.Apache.Reef.Common.Capabilities;
+
+namespace Org.Apache.Reef.Common.Catalog
+{
+    public interface INodeDescriptor
+    {
+        IPEndPoint InetSocketAddress { get; set; }
+
+        string HostName { get; set; }
+
+        CPU Cpu { get; set; }
+
+        RAM Ram { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/IRackDescriptor.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/IRackDescriptor.cs b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/IRackDescriptor.cs
new file mode 100644
index 0000000..ce576c4
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-common/ReefCommon/catalog/IRackDescriptor.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.Catalog
+{
+    public interface IRackDescriptor : IResourceCatalog
+    {
+    }
+}