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:42:51 UTC

[07/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/Tests/ReefTests/Functional.Tests/Driver/DriverTestStartHandler.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Functional.Tests/Driver/DriverTestStartHandler.cs b/lang/cs/Tests/ReefTests/Functional.Tests/Driver/DriverTestStartHandler.cs
new file mode 100644
index 0000000..4ab94dd
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Functional.Tests/Driver/DriverTestStartHandler.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.Driver;
+using Org.Apache.Reef.Driver.bridge;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Wake.Time;
+
+namespace Org.Apache.Reef.Test
+{
+    public class DriverTestStartHandler : IStartHandler
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(DriverTestStartHandler));
+
+        private IClock _clock;
+        private HttpServerPort _httpServerPort;
+
+        [Inject]
+        public DriverTestStartHandler(IClock clock, HttpServerPort httpServerPort)
+        {
+            _clock = clock;
+            _httpServerPort = httpServerPort;
+            Identifier = "DriverTestStartHandler";
+            LOGGER.Log(Level.Info, "Http Server port number: " + httpServerPort.PortNumber);
+        }
+
+        public string Identifier { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/Functional.Tests/Driver/TestDriver.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Functional.Tests/Driver/TestDriver.cs b/lang/cs/Tests/ReefTests/Functional.Tests/Driver/TestDriver.cs
new file mode 100644
index 0000000..8f7b36a
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Functional.Tests/Driver/TestDriver.cs
@@ -0,0 +1,68 @@
+/**
+ * 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.Collections.Generic;
+using Org.Apache.Reef.Driver.Bridge;
+using Org.Apache.Reef.Driver.Defaults;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Tang.Util;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+
+namespace Org.Apache.Reef.Test
+{
+    [TestClass]
+    public class TestDriver : ReefFunctionalTest
+    {
+        [TestInitialize()]
+        public void TestSetup()
+        {
+            CleanUp();
+        }
+
+        [TestCleanup]
+        public void TestCleanup()
+        {
+            CleanUp();
+        }
+
+        /// <summary>
+        /// This is to test DriverTestStartHandler. No evaluator and tasks are involked.
+        /// </summary>
+        [TestMethod, Priority(1), TestCategory("FunctionalGated")]
+        [Description("Test DriverTestStartHandler. No evaluator and tasks are involked")]
+        [DeploymentItem(@".")]
+        [Timeout(180 * 1000)]
+        public void TestDriverStart()
+        {
+            IConfiguration driverConfig = DriverBridgeConfiguration.ConfigurationModule
+             .Set(DriverBridgeConfiguration.OnDriverStarted, GenericType<DriverTestStartHandler>.Class)
+             .Set(DriverBridgeConfiguration.CustomTraceListeners, GenericType<DefaultCustomTraceListener>.Class)
+             .Set(DriverBridgeConfiguration.CustomTraceLevel, Level.Info.ToString())
+             .Build();
+
+            HashSet<string> appDlls = new HashSet<string>();
+            appDlls.Add(typeof(DriverTestStartHandler).Assembly.GetName().Name);
+
+            TestRun(appDlls, driverConfig);
+
+            ValidateSuccessForLocalRuntime(0);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageDriver.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageDriver.cs b/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageDriver.cs
new file mode 100644
index 0000000..c86c7ae
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageDriver.cs
@@ -0,0 +1,122 @@
+/**
+ * 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.Globalization;
+using System.Net;
+using System.Text;
+using Org.Apache.Reef.Driver;
+using Org.Apache.Reef.Driver.Bridge;
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Driver.Evaluator;
+using Org.Apache.Reef.Driver.Task;
+using Org.Apache.Reef.IO.Network.Naming;
+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.Annotations;
+using Org.Apache.Reef.Tang.Implementations.Configuration;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Tang.Util;
+
+namespace Org.Apache.Reef.Test
+{
+    public class MessageDriver : IStartHandler, IObserver<IAllocatedEvaluator>, IObserver<IEvaluatorRequestor>, IObserver<ITaskMessage>, IObserver<IRunningTask>
+    {
+        public const int NumerOfEvaluator = 1;
+
+        public const string Message = "MESSAGE::DRIVER";
+
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(MessageDriver));
+
+        [Inject]
+        public MessageDriver()
+        {
+            CreateClassHierarchy();
+            Identifier = "TaskMessagingStartHandler";
+        }
+
+        public string Identifier { get; set; }
+
+        public void OnNext(IAllocatedEvaluator eval)
+        {
+            string taskId = "Task_" + eval.Id;
+
+            IConfiguration contextConfiguration = ContextConfiguration.ConfigurationModule
+                .Set(ContextConfiguration.Identifier, taskId)
+                .Build();
+
+            IConfiguration taskConfiguration = TaskConfiguration.ConfigurationModule
+                .Set(TaskConfiguration.Identifier, taskId)
+                .Set(TaskConfiguration.Task, GenericType<MessageTask>.Class)
+                .Set(TaskConfiguration.OnMessage, GenericType<MessageTask.MessagingDriverMessageHandler>.Class)
+                .Set(TaskConfiguration.OnSendMessage, GenericType<MessageTask>.Class)
+                .Build();
+
+            eval.SubmitContextAndTask(contextConfiguration, taskConfiguration);
+        }
+
+        public void OnNext(IEvaluatorRequestor evalutorRequestor)
+        {
+            EvaluatorRequest request = new EvaluatorRequest(NumerOfEvaluator, 512, 2, "WonderlandRack", "TaskMessagingEvaluator");
+            evalutorRequestor.Submit(request);
+        }
+
+        public void OnNext(ITaskMessage taskMessage)
+        {
+            string msgReceived = ByteUtilities.ByteArrarysToString(taskMessage.Message);
+
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "CLR TaskMessagingTaskMessageHandler received following message from Task: {0}, Message: {1}.", taskMessage.TaskId, msgReceived));
+
+            if (!msgReceived.StartsWith(MessageTask.MessageSend, true, CultureInfo.CurrentCulture))
+            {
+                Exceptions.Throw(new Exception("Unexpected message: " + msgReceived), "Unexpected task message received: " + msgReceived, LOGGER);
+            }
+        }
+
+        public void OnNext(IRunningTask runningTask)
+        {
+            LOGGER.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "TaskMessegingRunningTaskHandler: {0} is to send message {1}.", runningTask.Id, Message));
+            runningTask.Send(ByteUtilities.StringToByteArrays(Message));
+        }
+
+        public void OnCompleted()
+        {
+            throw new NotImplementedException();
+        }
+
+        public void OnError(Exception error)
+        {
+            throw new NotImplementedException();
+        }
+
+        private void CreateClassHierarchy()
+        {
+            HashSet<string> clrDlls = new HashSet<string>();
+            clrDlls.Add(typeof(IDriver).Assembly.GetName().Name);
+            clrDlls.Add(typeof(ITask).Assembly.GetName().Name);
+            clrDlls.Add(typeof(MessageTask).Assembly.GetName().Name);
+
+            ClrHandlerHelper.GenerateClassHierarchy(clrDlls);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageTask.cs b/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageTask.cs
new file mode 100644
index 0000000..164724f
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/MessageTask.cs
@@ -0,0 +1,106 @@
+/**
+ * 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 System.Linq;
+using System.Net;
+using System.Threading;
+using Org.Apache.Reef.IO.Network.Naming;
+using Org.Apache.Reef.Tasks;
+using Org.Apache.Reef.Tasks.Events;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Test
+{
+    public class MessageTask : ITask, ITaskMessageSource
+    {
+        public const string MessageSend = "MESSAGE:TASK";
+
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(MessageTask));
+
+        [Inject]
+        public MessageTask()
+        {
+        }
+
+        public HelloService Service { get; set; }
+
+        public Optional<TaskMessage> Message
+        {
+            get
+            {
+                TaskMessage defaultTaskMessage = TaskMessage.From(
+                    "messagingSourceId",
+                    ByteUtilities.StringToByteArrays(MessageSend + " generated at " + DateTime.Now.ToString(CultureInfo.InvariantCulture)));
+                return Optional<TaskMessage>.Of(defaultTaskMessage);
+            }
+
+            set
+            {
+            }
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            Console.WriteLine("Hello, CLR TaskMsg!");
+            Thread.Sleep(5 * 1000);
+            return null;
+        }
+
+        public void Dispose()
+        {
+            LOGGER.Log(Level.Info, "TaskMsg disposed.");
+        }
+
+        private void DriverMessage(string message)
+        {
+            LOGGER.Log(Level.Info, "Receieved DriverMessage in TaskMsg: " + message);
+            if (!message.Equals(MessageDriver.Message))
+            {
+                Exceptions.Throw(new Exception("Unexpected driver message: " + message), "Unexpected driver message received: " + message, LOGGER);
+            }
+        }
+
+        public class MessagingDriverMessageHandler : IDriverMessageHandler
+        {
+            private MessageTask _parentTask;
+
+            [Inject]
+            public MessagingDriverMessageHandler(MessageTask task)
+            {
+                _parentTask = task;
+            }
+
+            public void Handle(IDriverMessage value)
+            {
+                string message = string.Empty;
+                LOGGER.Log(Level.Verbose, "Receieved a message from driver, handling it with MessagingDriverMessageHandler");
+                if (value.Message.IsPresent())
+                {
+                    message = ByteUtilities.ByteArrarysToString(value.Message.Value);
+                }
+                _parentTask.DriverMessage(message);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/TestTaskMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/TestTaskMessage.cs b/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/TestTaskMessage.cs
new file mode 100644
index 0000000..89c6fc3
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Functional.Tests/Messaging/TestTaskMessage.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 System.Collections.Generic;
+using Org.Apache.Reef.Driver.Bridge;
+using Org.Apache.Reef.Driver.Defaults;
+using Org.Apache.Reef.Tasks;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Tang.Util;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+
+namespace Org.Apache.Reef.Test
+{
+    [TestClass]
+    public class TestTaskMessage : ReefFunctionalTest
+    {
+        [TestInitialize()]
+        public void TestSetup()
+        {
+            CleanUp();
+            Init();
+        }
+
+        [TestCleanup]
+        public void TestCleanup()
+        {
+            CleanUp();
+        }
+
+        /// <summary>
+        /// This test is to test both task message and driver message. The messages are sent 
+        /// from one side and received in the corresponding handlers and verified in the test 
+        /// </summary>
+        [TestMethod, Priority(1), TestCategory("FunctionalGated")]
+        [Description("Test task message and driver message")]
+        [DeploymentItem(@".")]
+        [Timeout(180 * 1000)]
+        public void TestSendTaskMessage()
+        {
+            IConfiguration driverConfig = DriverBridgeConfiguration.ConfigurationModule
+             .Set(DriverBridgeConfiguration.OnDriverStarted, GenericType<MessageDriver>.Class)
+             .Set(DriverBridgeConfiguration.OnEvaluatorAllocated, GenericType<MessageDriver>.Class)
+             .Set(DriverBridgeConfiguration.OnTaskMessage, GenericType<MessageDriver>.Class)
+             .Set(DriverBridgeConfiguration.OnTaskRunning, GenericType<MessageDriver>.Class)
+             .Set(DriverBridgeConfiguration.OnEvaluatorRequested, GenericType<MessageDriver>.Class)
+             .Set(DriverBridgeConfiguration.CustomTraceListeners, GenericType<DefaultCustomTraceListener>.Class)
+             .Set(DriverBridgeConfiguration.CustomTraceLevel, Level.Info.ToString())
+             .Build();
+
+            HashSet<string> appDlls = new HashSet<string>();
+            appDlls.Add(typeof(MessageDriver).Assembly.GetName().Name);
+            appDlls.Add(typeof(MessageTask).Assembly.GetName().Name);
+
+            TestRun(appDlls, driverConfig);
+            ValidateSuccessForLocalRuntime(MessageDriver.NumerOfEvaluator);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/Functional.Tests/ReefFunctionalTest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Functional.Tests/ReefFunctionalTest.cs b/lang/cs/Tests/ReefTests/Functional.Tests/ReefFunctionalTest.cs
new file mode 100644
index 0000000..0dd2b46
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Functional.Tests/ReefFunctionalTest.cs
@@ -0,0 +1,200 @@
+/**
+ * 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.Globalization;
+using System.IO;
+using System.Linq;
+using System.Threading;
+using System.Threading.Tasks;
+using System.Timers;
+using Org.Apache.Reef.Driver;
+using Org.Apache.Reef.Driver.bridge;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Interface;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Blob;
+using Timer = System.Timers.Timer;
+
+namespace Org.Apache.Reef.Test
+{
+    public class ReefFunctionalTest
+    {
+        protected const string _stdout = "driver.stdout";
+        protected const string _stderr = "driver.stderr";
+        protected const string _cmdFile = "run.cmd";
+        protected const string _binFolder = "bin";
+        // TODO: we will need a proper way to hide this when we are OSS'ed
+        protected const string _blobStorageConnectionString =
+            @"DefaultEndpointsProtocol=https;AccountName=reeftestlog;AccountKey=cuUmPRF9DiG56bciNc37O/SfHAoh1jFfJW6AsXAtWLPnjlOzXJGWgXhkyDFOGEHIMscqDU41ElUKnjcsJjWD9w==";
+
+        private bool _testSuccess = false;
+        private bool _onLocalRuntime = false;
+        private string _className = Constants.BridgeLaunchClass;
+        private string _clrFolder = ".";
+        private string _reefJar = Path.Combine(_binFolder, Constants.BridgeJarFileName);
+        private string _runCommand = Path.Combine(_binFolder, _cmdFile);
+
+        // TODO: once things stablize, we would like to toggle this to be false and only enable when needed for debugging test failures.
+        private bool _enableRealtimeLogUpload = true; 
+
+        protected string TestId { get; set; }
+
+        protected Timer TestTimer { get; set; }
+
+        protected Task TimerTask { get; set; }
+
+        protected bool TestSuccess 
+        {
+            get { return _testSuccess; }
+            set { _testSuccess = value; }
+        }
+
+        protected bool IsOnLocalRuntiime
+        {
+            get { return _onLocalRuntime; }
+            set { _onLocalRuntime = value; }
+        }
+
+        public void Init()
+        {
+            TestId = Guid.NewGuid().ToString("N").Substring(0, 8);
+            Console.WriteLine("Running test " + TestId + ". If failed AND log uploaded is enabled, log can be find in " + Path.Combine(DateTime.Now.ToString("yyyy-MM-dd", CultureInfo.InvariantCulture), TestId));
+            if (_enableRealtimeLogUpload)
+            {
+                TimerTask = new Task(() =>
+                {
+                    TestTimer = new Timer()
+                    {
+                        Interval = 1000,
+                        Enabled = true,
+                        AutoReset = true
+                    };
+                    TestTimer.Elapsed += PeriodicUploadLog;
+                    TestTimer.Start();
+                });
+                TimerTask.Start(); 
+            }
+            
+            ValidationUtilities.ValidateEnvVariable("JAVA_HOME");
+
+            if (!Directory.Exists(_binFolder))
+            {
+                throw new InvalidOperationException(_binFolder + " not found in current directory, cannot init test");
+            }
+        }
+
+        protected void TestRun(HashSet<string> appDlls, IConfiguration driverBridgeConfig, bool runOnYarn = false, JavaLoggingSetting javaLogSettings = JavaLoggingSetting.INFO)
+        {
+            ClrClientHelper.Run(appDlls, driverBridgeConfig, new DriverSubmissionSettings() { RunOnYarn = runOnYarn, JavaLogLevel = javaLogSettings }, _reefJar, _runCommand, _clrFolder, _className);
+        }
+
+        protected void CleanUp()
+        {
+            Console.WriteLine("Cleaning up test.");
+
+            if (TimerTask != null)
+            {
+                TestTimer.Stop();
+                TimerTask.Dispose();
+                TimerTask = null;
+            }
+            
+            // Wait for file upload task to complete
+            Thread.Sleep(500);
+
+            string dir = Path.Combine(Directory.GetCurrentDirectory(), "REEF_LOCAL_RUNTIME");
+            try
+            {
+                if (Directory.Exists(dir))
+                {
+                    Directory.Delete(dir, true);
+                }
+            }
+            catch (IOException)
+            {
+                // do not fail if clean up is unsuccessful
+            }   
+        }
+
+        protected void ValidateSuccessForLocalRuntime(int numberOfEvaluatorsToClose)
+        {
+            const string successIndication = "EXIT: ActiveContextClr2Java::Close";
+            const string failedTaskIndication = "Java_com_microsoft_reef_javabridge_NativeInterop_ClrSystemFailedTaskHandlerOnNext";
+            const string failedEvaluatorIndication = "Java_com_microsoft_reef_javabridge_NativeInterop_ClrSystemFailedEvaluatorHandlerOnNext";
+            string[] lines = File.ReadAllLines(GetLogFile(_stdout));
+            string[] successIndicators = lines.Where(s => s.Contains(successIndication)).ToArray();
+            string[] failedTaskIndicators = lines.Where(s => s.Contains(failedTaskIndication)).ToArray();
+            string[] failedIndicators = lines.Where(s => s.Contains(failedEvaluatorIndication)).ToArray();
+            Assert.IsTrue(successIndicators.Count() == numberOfEvaluatorsToClose);
+            Assert.IsFalse(failedTaskIndicators.Any());
+            Assert.IsFalse(failedIndicators.Any());
+        }
+
+        protected void PeriodicUploadLog(object source, ElapsedEventArgs e)
+        {
+            try
+            {
+                UploadDriverLog();
+            }
+            catch (Exception)
+            {
+                // log not available yet, ignore it
+            }
+        }
+
+        protected string GetLogFile(string logFileName)
+        {
+            string driverContainerDirectory = Directory.GetDirectories(Path.Combine(Directory.GetCurrentDirectory(), "REEF_LOCAL_RUNTIME"), "driver", SearchOption.AllDirectories).SingleOrDefault();
+            if (string.IsNullOrWhiteSpace(driverContainerDirectory))
+            {
+                throw new InvalidOperationException("Cannot find driver container directory");
+            }
+            string logFile = Path.Combine(driverContainerDirectory, logFileName);
+            if (!File.Exists(logFile))
+            {
+                throw new InvalidOperationException("Driver stdout file not found");
+            }
+            return logFile;
+        }
+
+        private void UploadDriverLog()
+        {
+            string driverStdout = GetLogFile(_stdout);
+            string driverStderr = GetLogFile(_stderr);
+            CloudStorageAccount storageAccount = CloudStorageAccount.Parse(_blobStorageConnectionString);
+            CloudBlobClient blobClient = storageAccount.CreateCloudBlobClient();
+            CloudBlobContainer container = blobClient.GetContainerReference(DateTime.Now.ToString("yyyy-MM-dd", CultureInfo.InvariantCulture));   
+            container.CreateIfNotExists();
+
+            CloudBlockBlob blob = container.GetBlockBlobReference(Path.Combine(TestId, "driverStdOut"));
+            FileStream fs = new FileStream(driverStdout, FileMode.Open, FileAccess.Read, FileShare.ReadWrite);
+            blob.UploadFromStream(fs);
+            fs.Close();
+
+            blob = container.GetBlockBlobReference(Path.Combine(TestId, "driverStdErr"));
+            fs = new FileStream(driverStderr, FileMode.Open, FileAccess.Read, FileShare.ReadWrite);
+            blob.UploadFromStream(fs);
+            fs.Close();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/IO.Tests/BlockingCollectionExtensionTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/IO.Tests/BlockingCollectionExtensionTests.cs b/lang/cs/Tests/ReefTests/IO.Tests/BlockingCollectionExtensionTests.cs
new file mode 100644
index 0000000..3ccd3e9
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/IO.Tests/BlockingCollectionExtensionTests.cs
@@ -0,0 +1,78 @@
+/**
+ * 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.IO.Network.Utilities;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+
+namespace Org.Apache.Reef.Test.IO.Tests
+{
+    [TestClass]
+    public class BlockingCollectionExtensionTests
+    {
+        [TestMethod]
+        public void TestCollectionContainsElement()
+        {
+            string item = "abc";
+            BlockingCollection<string> collection = new BlockingCollection<string>();
+            collection.Add(item);
+
+            Assert.AreEqual(item, collection.Take(item));
+
+            // Check that item is no longer in collection
+            Assert.AreEqual(0, collection.Count);
+        }
+
+        [TestMethod]
+        public void TestCollectionContainsElement2()
+        {
+            string item = "abc";
+            BlockingCollection<string> collection = new BlockingCollection<string>();
+            collection.Add("cat");
+            collection.Add(item);
+            collection.Add("dog");
+
+            Assert.AreEqual(item, collection.Take(item));
+
+            // Remove remaining items, check that item is not there
+            Assert.AreNotEqual(item, collection.Take());
+            Assert.AreNotEqual(item, collection.Take());
+            Assert.AreEqual(0, collection.Count);
+        }
+
+        [TestMethod]
+        [ExpectedException(typeof(InvalidOperationException))]
+        public void TestCollectionDoesNotContainsElement()
+        {
+            string item1 = "abc";
+            string item2 = "def";
+
+            BlockingCollection<string> collection = new BlockingCollection<string>();
+            collection.Add(item2);
+
+            // Should throw InvalidOperationException since item1 is not in collection
+            collection.Take(item1);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/IO.Tests/NameServerTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/IO.Tests/NameServerTests.cs b/lang/cs/Tests/ReefTests/IO.Tests/NameServerTests.cs
new file mode 100644
index 0000000..ee118da
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/IO.Tests/NameServerTests.cs
@@ -0,0 +1,274 @@
+/**
+ * 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.Linq;
+using System.Net;
+using System.Text;
+using System.Threading;
+using System.Threading.Tasks;
+using Org.Apache.Reef.Common.io;
+using Org.Apache.Reef.IO.Network.Naming;
+using Org.Apache.Reef.IO.Network.Naming.Events;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Tang.Implementations;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Tang.Util;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Org.Apache.Reef.Wake.Util;
+
+namespace Org.Apache.Reef.Test
+{
+    [TestClass]
+    public class NameServerTests
+    {
+        [TestMethod]
+        public void TestNameServerNoRequests()
+        {
+           using (var server = new NameServer(0))
+           {
+           }
+        }
+
+        [TestMethod]
+        public void TestNameServerNoRequestsTwoClients()
+        {
+           using (var server = new NameServer(0))
+           {
+               var nameClient = new NameClient(server.LocalEndpoint);
+               var nameClient2 = new NameClient(server.LocalEndpoint);
+               nameClient2.Register("1", new IPEndPoint(IPAddress.Any, 8080));
+               nameClient.Lookup("1");
+           }
+        }
+
+        [TestMethod]
+        public void TestNameServerNoRequestsTwoClients2()
+        {
+           using (var server = new NameServer(0))
+           {
+               var nameClient = new NameClient(server.LocalEndpoint);
+               var nameClient2 = new NameClient(server.LocalEndpoint);
+               nameClient2.Register("1", new IPEndPoint(IPAddress.Any, 8080));
+               nameClient.Lookup("1");
+           }
+        }
+
+        [TestMethod]
+        public void TestNameServerMultipleRequestsTwoClients()
+        {
+           using (var server = new NameServer(0))
+           {
+               var nameClient = new NameClient(server.LocalEndpoint);
+               var nameClient2 = new NameClient(server.LocalEndpoint);
+               nameClient.Register("1", new IPEndPoint(IPAddress.Any, 8080));
+               nameClient2.Lookup("1");
+           }
+        }
+
+        [TestMethod]
+        public void TestRegister()
+        {
+            using (INameServer server = BuildNameServer())
+            {
+                using (INameClient client = BuildNameClient(server.LocalEndpoint))
+                {
+                    IPEndPoint endpoint1 = new IPEndPoint(IPAddress.Parse("100.0.0.1"), 100);
+                    IPEndPoint endpoint2 = new IPEndPoint(IPAddress.Parse("100.0.0.2"), 200);
+                    IPEndPoint endpoint3 = new IPEndPoint(IPAddress.Parse("100.0.0.3"), 300);
+
+                    // Check that no endpoints have been registered
+                    Assert.IsNull(client.Lookup("a"));
+                    Assert.IsNull(client.Lookup("b"));
+                    Assert.IsNull(client.Lookup("c"));
+                
+                    // Register endpoints
+                    client.Register("a", endpoint1);
+                    client.Register("b", endpoint2);
+                    client.Register("c", endpoint3);
+
+                    // Check that they can be looked up correctly
+                    Assert.AreEqual(endpoint1, client.Lookup("a"));
+                    Assert.AreEqual(endpoint2, client.Lookup("b"));
+                    Assert.AreEqual(endpoint3, client.Lookup("c"));
+                }
+            }
+        }
+
+        [TestMethod]
+        public void TestUnregister()
+        {
+            using (INameServer server = BuildNameServer())
+            {
+                using (INameClient client = BuildNameClient(server.LocalEndpoint))
+                {
+                    IPEndPoint endpoint1 = new IPEndPoint(IPAddress.Parse("100.0.0.1"), 100);
+                
+                    // Register endpoint
+                    client.Register("a", endpoint1);
+
+                    // Check that it can be looked up correctly
+                    Assert.AreEqual(endpoint1, client.Lookup("a"));
+
+                    // Unregister endpoints
+                    client.Unregister("a");
+                    Thread.Sleep(1000);
+
+                    // Make sure they were unregistered correctly
+                    Assert.IsNull(client.Lookup("a"));
+                }
+            }
+        }
+
+        [TestMethod]
+        public void TestLookup()
+        {
+            using (INameServer server = BuildNameServer())
+            {
+                using (INameClient client = BuildNameClient(server.LocalEndpoint))
+                {
+                    IPEndPoint endpoint1 = new IPEndPoint(IPAddress.Parse("100.0.0.1"), 100);
+                    IPEndPoint endpoint2 = new IPEndPoint(IPAddress.Parse("100.0.0.2"), 200);
+                
+                    // Register endpoint1
+                    client.Register("a", endpoint1);
+                    Assert.AreEqual(endpoint1, client.Lookup("a"));
+
+                    // Reregister identifer a
+                    client.Register("a", endpoint2);
+                    Assert.AreEqual(endpoint2, client.Lookup("a"));
+                }
+            }
+        }
+
+        [TestMethod]
+        public void TestLookupList()
+        {
+            using (INameServer server = BuildNameServer())
+            {
+                using (INameClient client = BuildNameClient(server.LocalEndpoint))
+                {
+                    IPEndPoint endpoint1 = new IPEndPoint(IPAddress.Parse("100.0.0.1"), 100);
+                    IPEndPoint endpoint2 = new IPEndPoint(IPAddress.Parse("100.0.0.2"), 200);
+                    IPEndPoint endpoint3 = new IPEndPoint(IPAddress.Parse("100.0.0.3"), 300);
+                
+                    // Register endpoints
+                    client.Register("a", endpoint1);
+                    client.Register("b", endpoint2);
+                    client.Register("c", endpoint3);
+
+                    // Look up both at the same time
+                    List<string> ids = new List<string> { "a", "b", "c", "d" };
+                    List<NameAssignment> assignments = client.Lookup(ids);
+
+                    // Check that a, b, and c are registered
+                    Assert.AreEqual("a", assignments[0].Identifier);
+                    Assert.AreEqual(endpoint1, assignments[0].Endpoint);
+                    Assert.AreEqual("b", assignments[1].Identifier);
+                    Assert.AreEqual(endpoint2, assignments[1].Endpoint);
+                    Assert.AreEqual("c", assignments[2].Identifier);
+                    Assert.AreEqual(endpoint3, assignments[2].Endpoint);
+
+                    // Check that d is not registered
+                    Assert.AreEqual(3, assignments.Count);
+                }
+            }
+        }
+
+        [TestMethod]
+        public void TestNameClientRestart()
+        {
+            int oldPort = 6666;
+            int newPort = 6662;
+            INameServer server = new NameServer(oldPort);
+
+            using (INameClient client = BuildNameClient(server.LocalEndpoint))
+            {
+                IPEndPoint endpoint = new IPEndPoint(IPAddress.Parse("100.0.0.1"), 100);
+            
+                client.Register("a", endpoint);
+                Assert.AreEqual(endpoint, client.Lookup("a"));
+
+                server.Dispose();
+
+                server = new NameServer(newPort);
+                client.Restart(server.LocalEndpoint);
+
+                client.Register("b", endpoint);
+                Assert.AreEqual(endpoint, client.Lookup("b"));
+
+                server.Dispose();
+            }
+        }
+
+        [TestMethod]
+        public void TestConstructorInjection()
+        {
+            int port = 6666;
+            using (INameServer server = new NameServer(port))
+            {
+                IConfiguration nameClientConfiguration = NamingConfiguration.ConfigurationModule
+                    .Set(NamingConfiguration.NameServerAddress, server.LocalEndpoint.Address.ToString())
+                    .Set(NamingConfiguration.NameServerPort, port + string.Empty)
+                    .Build();
+
+                ConstructorInjection c = TangFactory.GetTang()
+                    .NewInjector(nameClientConfiguration)
+                    .GetInstance<ConstructorInjection>();
+
+                Assert.IsNotNull(c);
+            }
+        }
+
+        private INameServer BuildNameServer()
+        {
+            var builder = TangFactory.GetTang()
+                                     .NewConfigurationBuilder()
+                                     .BindNamedParameter<NamingConfigurationOptions.NameServerPort, int>(
+                                         GenericType<NamingConfigurationOptions.NameServerPort>.Class, "0");
+
+            return TangFactory.GetTang().NewInjector(builder.Build()).GetInstance<INameServer>();
+        }
+
+        private INameClient BuildNameClient(IPEndPoint remoteEndpoint)
+        {
+            string nameServerAddr = remoteEndpoint.Address.ToString();
+            int nameServerPort = remoteEndpoint.Port;
+            IConfiguration nameClientConfiguration = NamingConfiguration.ConfigurationModule
+                .Set(NamingConfiguration.NameServerAddress, nameServerAddr)
+                .Set(NamingConfiguration.NameServerPort, nameServerPort + string.Empty)
+                .Build();
+
+            return TangFactory.GetTang().NewInjector(nameClientConfiguration).GetInstance<NameClient>();
+        }
+
+        private class ConstructorInjection
+        {
+            [Inject]
+            public ConstructorInjection(NameClient client)
+            {
+                if (client == null)
+                {
+                    throw new ArgumentNullException("client");
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/IO.Tests/NetworkServiceTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/IO.Tests/NetworkServiceTests.cs b/lang/cs/Tests/ReefTests/IO.Tests/NetworkServiceTests.cs
new file mode 100644
index 0000000..46dce1b
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/IO.Tests/NetworkServiceTests.cs
@@ -0,0 +1,202 @@
+/**
+ * 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.io;
+using Org.Apache.Reef.IO.Network.Naming;
+using Org.Apache.Reef.IO.Network.NetworkService;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Tang.Implementations;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Tang.Util;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Org.Apache.Reef.Wake;
+using Org.Apache.Reef.Wake.Remote;
+using Org.Apache.Reef.Wake.Remote.Impl;
+using Org.Apache.Reef.Wake.Util;
+using System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+using System.Globalization;
+using System.Linq;
+using System.Net;
+using System.Text;
+using System.Threading.Tasks;
+
+namespace Org.Apache.Reef.Test.IO.Tests
+{
+    [TestClass]
+    public class NetworkServiceTests
+    {
+        [TestMethod]
+        public void TestNetworkServiceOneWayCommunication()
+        {
+            int networkServicePort1 = NetworkUtils.GenerateRandomPort(6000, 7000);
+            int networkServicePort2 = NetworkUtils.GenerateRandomPort(7001, 8000);
+
+            BlockingCollection<string> queue = new BlockingCollection<string>();
+
+            using (INameServer nameServer = new NameServer(0))
+            {
+                IPEndPoint endpoint = nameServer.LocalEndpoint;
+                int nameServerPort = endpoint.Port;
+                string nameServerAddr = endpoint.Address.ToString();
+                using (INetworkService<string> networkService1 = BuildNetworkService(networkServicePort1, nameServerPort, nameServerAddr, null))
+                using (INetworkService<string> networkService2 = BuildNetworkService(networkServicePort2, nameServerPort, nameServerAddr, new MessageHandler(queue)))
+                {
+                    IIdentifier id1 = new StringIdentifier("service1");
+                    IIdentifier id2 = new StringIdentifier("service2");
+                    networkService1.Register(id1);
+                    networkService2.Register(id2);
+
+                    using (IConnection<string> connection = networkService1.NewConnection(id2))
+                    {
+                        connection.Open();
+                        connection.Write("abc");
+                        connection.Write("def");
+                        connection.Write("ghi");
+
+                        Assert.AreEqual("abc", queue.Take());
+                        Assert.AreEqual("def", queue.Take());
+                        Assert.AreEqual("ghi", queue.Take());
+                    }
+                }
+            }
+        }
+
+        [TestMethod]
+        public void TestNetworkServiceTwoWayCommunication()
+        {
+            int networkServicePort1 = NetworkUtils.GenerateRandomPort(6000, 7000);
+            int networkServicePort2 = NetworkUtils.GenerateRandomPort(7001, 8000);
+
+            BlockingCollection<string> queue1 = new BlockingCollection<string>();
+            BlockingCollection<string> queue2 = new BlockingCollection<string>();
+
+            using (INameServer nameServer = new NameServer(0))
+            {
+                IPEndPoint endpoint = nameServer.LocalEndpoint;
+                int nameServerPort = endpoint.Port;
+                string nameServerAddr = endpoint.Address.ToString();
+                using (INetworkService<string> networkService1 = BuildNetworkService(networkServicePort1, nameServerPort, nameServerAddr, new MessageHandler(queue1)))
+                using (INetworkService<string> networkService2 = BuildNetworkService(networkServicePort2, nameServerPort, nameServerAddr, new MessageHandler(queue2)))
+                {
+                    IIdentifier id1 = new StringIdentifier("service1");
+                    IIdentifier id2 = new StringIdentifier("service2");
+                    networkService1.Register(id1);
+                    networkService2.Register(id2);
+
+                    using (IConnection<string> connection1 = networkService1.NewConnection(id2))
+                    using (IConnection<string> connection2 = networkService2.NewConnection(id1))
+                    {
+                        connection1.Open();
+                        connection1.Write("abc");
+                        connection1.Write("def");
+                        connection1.Write("ghi");
+
+                        connection2.Open();
+                        connection2.Write("jkl");
+                        connection2.Write("mno");
+
+                        Assert.AreEqual("abc", queue2.Take());
+                        Assert.AreEqual("def", queue2.Take());
+                        Assert.AreEqual("ghi", queue2.Take());
+
+                        Assert.AreEqual("jkl", queue1.Take());
+                        Assert.AreEqual("mno", queue1.Take());
+                    }
+                }
+            }
+        }
+
+        private INetworkService<string> BuildNetworkService(
+            int networkServicePort,
+            int nameServicePort,
+            string nameServiceAddr,
+            IObserver<NsMessage<string>> handler)
+        {
+            // Test injection
+            if (handler == null)
+            {
+                var networkServiceConf = TangFactory.GetTang().NewConfigurationBuilder()
+                    .BindNamedParameter<NetworkServiceOptions.NetworkServicePort, int>(
+                        GenericType<NetworkServiceOptions.NetworkServicePort>.Class,
+                        networkServicePort.ToString(CultureInfo.CurrentCulture))
+                    .BindNamedParameter<NamingConfigurationOptions.NameServerPort, int>(
+                        GenericType<NamingConfigurationOptions.NameServerPort>.Class,
+                        nameServicePort.ToString(CultureInfo.CurrentCulture))
+                    .BindNamedParameter<NamingConfigurationOptions.NameServerAddress, string>(
+                        GenericType<NamingConfigurationOptions.NameServerAddress>.Class,
+                        nameServiceAddr)
+                    .BindImplementation(GenericType<ICodec<string>>.Class, GenericType<StringCodec>.Class)
+                    .BindImplementation(GenericType<IObserver<NsMessage<string>>>.Class, GenericType<NetworkMessageHandler>.Class)
+                    .Build();
+
+                return TangFactory.GetTang().NewInjector(networkServiceConf).GetInstance<NetworkService<string>>();
+            }
+
+            return new NetworkService<string>(networkServicePort, nameServiceAddr, nameServicePort, 
+                handler, new StringIdentifierFactory(), new StringCodec());
+        }
+
+        private class MessageHandler : IObserver<NsMessage<string>>
+        {
+            private BlockingCollection<string> _queue;
+
+            public MessageHandler(BlockingCollection<string> queue)
+            {
+                _queue = queue;
+            }
+
+            public void OnNext(NsMessage<string> value)
+            {
+                _queue.Add(value.Data.First());
+            }
+
+            public void OnError(Exception error)
+            {
+                throw new NotImplementedException();
+            }
+
+            public void OnCompleted()
+            {
+                throw new NotImplementedException();
+            }
+        }
+
+        private class NetworkMessageHandler : IObserver<NsMessage<string>>
+        {
+            [Inject]
+            public NetworkMessageHandler()
+            {
+            }
+
+            public void OnNext(NsMessage<string> value)
+            {
+            }
+
+            public void OnError(Exception error)
+            {
+            }
+
+            public void OnCompleted()
+            {
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Properties/AssemblyInfo.cs b/lang/cs/Tests/ReefTests/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..9d92b89
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/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("ReefTests")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("ReefTests")]
+[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("19ebceb4-3b1f-466b-9127-aa14e636d723")]
+
+// 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/Tests/ReefTests/ReefTests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/ReefTests.csproj b/lang/cs/Tests/ReefTests/ReefTests.csproj
new file mode 100644
index 0000000..221dd0b
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/ReefTests.csproj
@@ -0,0 +1,192 @@
+<?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>{988F90CF-A48D-4938-A4D2-FA3B758FB5A7}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Org.Apache.Reef.Test</RootNamespace>
+    <AssemblyName>Org.Apache.Reef.Test</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.Test\</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.Tests\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="Microsoft.Data.Edm, Version=5.6.3.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <SpecificVersion>False</SpecificVersion>
+      <HintPath>..\..\packages\Microsoft.Data.Edm.5.6.3\lib\net40\Microsoft.Data.Edm.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.Data.OData, Version=5.6.3.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <SpecificVersion>False</SpecificVersion>
+      <HintPath>..\..\packages\Microsoft.Data.OData.5.6.3\lib\net40\Microsoft.Data.OData.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.Data.Services.Client, Version=5.6.3.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <SpecificVersion>False</SpecificVersion>
+      <HintPath>..\..\packages\Microsoft.Data.Services.Client.5.6.3\lib\net40\Microsoft.Data.Services.Client.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.Hadoop.Avro">
+      <HintPath>..\..\packages\Microsoft.Hadoop.Avro.1.4.0.0\lib\net40\Microsoft.Hadoop.Avro.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.VisualStudio.QualityTools.UnitTestFramework, Version=10.1.0.0, Culture=neutral, PublicKeyToken=b03f5f7f11d50a3a, processorArchitecture=MSIL" />
+    <Reference Include="Microsoft.WindowsAzure.Configuration">
+      <HintPath>..\..\packages\Microsoft.WindowsAzure.ConfigurationManager.2.0.3\lib\net40\Microsoft.WindowsAzure.Configuration.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.WindowsAzure.Storage">
+      <HintPath>..\..\packages\WindowsAzure.Storage.4.3.0\lib\net40\Microsoft.WindowsAzure.Storage.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.Spatial, Version=5.6.3.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <SpecificVersion>False</SpecificVersion>
+      <HintPath>..\..\packages\System.Spatial.5.6.3\lib\net40\System.Spatial.dll</HintPath>
+    </Reference>
+    <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.Tests\EvaluatorConfigurationsTests.cs" />
+    <Compile Include="Evaluator.Tests\EvaluatorTests.cs" />
+    <Compile Include="Functional.Tests\Bridge\TestBridgeClient.cs" />
+    <Compile Include="Functional.Tests\Bridge\TestHelloBridgeHandlers.cs" />
+    <Compile Include="Functional.Tests\Bridge\TestSimpleEventHandlers.cs" />
+    <Compile Include="Functional.Tests\Driver\DriverTestStartHandler.cs" />
+    <Compile Include="Functional.Tests\Driver\TestDriver.cs" />
+    <Compile Include="Functional.Tests\Messaging\MessageDriver.cs" />
+    <Compile Include="Functional.Tests\Messaging\MessageTask.cs" />
+    <Compile Include="Functional.Tests\Messaging\TestTaskMessage.cs" />
+    <Compile Include="Functional.Tests\ReefFunctionalTest.cs" />
+    <Compile Include="IO.Tests\BlockingCollectionExtensionTests.cs" />
+    <Compile Include="IO.Tests\NameServerTests.cs" />
+    <Compile Include="IO.Tests\NetworkServiceTests.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="Utility.Test\TestDriverConfigGenerator.cs" />
+    <Compile Include="Utility.Test\TestExceptions.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="bin\run.cmd">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </None>
+    <None Include="ConfigFiles\evaluator.conf">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </None>
+    <None Include="packages.config" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\..\Source\REEF\reef-applications\CLRBridgeClient\CLRBridgeClient.csproj">
+      <Project>{5094c35b-4fdb-4322-ac05-45d684501cbf}</Project>
+      <Name>CLRBridgeClient</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\REEF\reef-applications\Evaluator\Evaluator.csproj">
+      <Project>{1b983182-9c30-464c-948d-f87eb93a8240}</Project>
+      <Name>Evaluator</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\REEF\reef-common\ReefCommon\ReefCommon.csproj">
+      <Project>{545a0582-4105-44ce-b99c-b1379514a630}</Project>
+      <Name>ReefCommon</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\REEF\reef-common\ReefDriver\ReefDriver.csproj">
+      <Project>{a6baa2a7-f52f-4329-884e-1bcf711d6805}</Project>
+      <Name>ReefDriver</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\REEF\reef-examples\HelloCLRBridge\HelloCLRBridge.csproj">
+      <Project>{a78dd8e8-31d0-4506-8738-daa9da86d55b}</Project>
+      <Name>HelloCLRBridge</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\REEF\reef-io\NetWork\NetWork.csproj">
+      <Project>{883ce800-6a6a-4e0a-b7fe-c054f4f2c1dc}</Project>
+      <Name>NetWork</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\REEF\reef-tasks\Tasks\Tasks.csproj">
+      <Project>{75503f90-7b82-4762-9997-94b5c68f15db}</Project>
+      <Name>Tasks</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\TANG\Tang\Tang.csproj">
+      <Project>{97dbb573-3994-417a-9f69-ffa25f00d2a6}</Project>
+      <Name>Tang</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\Utilities\Utilities.csproj">
+      <Project>{79e7f89a-1dfb-45e1-8d43-d71a954aeb98}</Project>
+      <Name>Utilities</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\..\Source\WAKE\Wake\Wake.csproj">
+      <Project>{cdfb3464-4041-42b1-9271-83af24cd5008}</Project>
+      <Name>Wake</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup>
+    <Content Include="bin\reef-bridge-0.11.0-incubating-SNAPSHOT-shaded.jar">
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </Content>
+  </ItemGroup>
+  <ItemGroup>
+    <Folder Include="bin\Debug\" />
+    <Folder Include="bin\Release\" />
+  </ItemGroup>
+  <ItemGroup>
+    <Service Include="{82A7F48D-3B50-4B1E-B82E-3ADA8210C358}" />
+  </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/Tests/ReefTests/Utility.Test/TestDriverConfigGenerator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Utility.Test/TestDriverConfigGenerator.cs b/lang/cs/Tests/ReefTests/Utility.Test/TestDriverConfigGenerator.cs
new file mode 100644
index 0000000..214188a
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Utility.Test/TestDriverConfigGenerator.cs
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using Org.Apache.Reef.Driver;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+
+namespace Org.Apache.Reef.Test.Utility.Test
+{
+    [TestClass]
+    public class TestDriverConfigGenerator
+    {
+        [TestMethod]
+        public void TestGeneratingFullDriverConfigFile()
+        {
+            DriverConfigurationSettings driverSubmissionSettings = new DriverConfigurationSettings()
+            {
+                DriverMemory = 1024,
+                DriverIdentifier = "juliaDriverId",
+                SubmissionDirectory = "reefClrBridgeTmp/job_" + Guid.NewGuid().ToString("N").Substring(0, 8),
+                IncludingHttpServer = true,
+                IncludingNameServer = true,
+                //ClrFolder = "C:\\Reef\\ReefApache\\incubator-reef\\reef-bridge-project\\reef-bridge\\dotnetHello",
+                ClrFolder = ".",
+                JarFileFolder = ".\\bin\\"
+            };
+
+            DriverConfigGenerator.DriverConfigurationBuilder(driverSubmissionSettings);
+        }
+
+        [TestMethod]
+        public void TestGeneratingDriverConfigFileWithoutHttp()
+        {
+            DriverConfigurationSettings driverSubmissionSettings = new DriverConfigurationSettings()
+            {
+                DriverMemory = 1024,
+                DriverIdentifier = "juliaDriverId",
+                SubmissionDirectory = "reefClrBridgeTmp/job_" + Guid.NewGuid().ToString("N").Substring(0, 8),
+                IncludingHttpServer = false,
+                IncludingNameServer = true,
+//                ClrFolder = "C:\\Reef\\ReefApache\\incubator-reef\\reef-bridge-project\\reef-bridge\\dotnetHello",
+                ClrFolder = ".",
+                JarFileFolder = ".\\bin\\"
+            };
+
+            DriverConfigGenerator.DriverConfigurationBuilder(driverSubmissionSettings);
+        }
+
+        [TestMethod]
+        public void TestGeneratingDriverConfigFileWithoutNameServer()
+        {
+            DriverConfigurationSettings driverSubmissionSettings = new DriverConfigurationSettings()
+            {
+                DriverMemory = 1024,
+                DriverIdentifier = "juliaDriverId",
+                SubmissionDirectory = "reefClrBridgeTmp/job_" + Guid.NewGuid().ToString("N").Substring(0, 8),
+                IncludingHttpServer = true,
+                IncludingNameServer = false,
+                //ClrFolder = "C:\\Reef\\ReefApache\\incubator-reef\\reef-bridge-project\\reef-bridge\\dotnetHello",
+                ClrFolder = ".",
+                JarFileFolder = ".\\bin\\"
+            };
+
+            DriverConfigGenerator.DriverConfigurationBuilder(driverSubmissionSettings);
+        }
+
+        [TestMethod]
+        public void TestGeneratingDriverConfigFileDriverOnly()
+        {
+            DriverConfigurationSettings driverSubmissionSettings = new DriverConfigurationSettings()
+            {
+                DriverMemory = 1024,
+                DriverIdentifier = "juliaDriverId",
+                SubmissionDirectory = "reefClrBridgeTmp/job_" + Guid.NewGuid().ToString("N").Substring(0, 8),
+                IncludingHttpServer = false,
+                IncludingNameServer = false,
+                //ClrFolder = "C:\\Reef\\ReefApache\\incubator-reef\\reef-bridge-project\\reef-bridge\\dotnetHello",
+                ClrFolder = ".",
+                JarFileFolder = ".\\bin\\"
+            };
+
+            DriverConfigGenerator.DriverConfigurationBuilder(driverSubmissionSettings);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/Utility.Test/TestExceptions.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/Utility.Test/TestExceptions.cs b/lang/cs/Tests/ReefTests/Utility.Test/TestExceptions.cs
new file mode 100644
index 0000000..16bde92
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/Utility.Test/TestExceptions.cs
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using Org.Apache.Reef.Utilities.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+
+namespace Org.Apache.Reef.Test.Utility.Test
+{
+    [TestClass]
+    public class TestExceptions
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(TestExceptions));
+
+        [TestMethod]
+        public void TestThrowCaught()
+        {
+            string msg = null;
+            try
+            {
+                Exceptions.Throw(new ApplicationException("test"), LOGGER);
+                msg = "not supposed to reach here";
+                Assert.Fail(msg);
+            }
+            catch (ApplicationException e)
+            {
+                Exceptions.Caught(e, Level.Info, LOGGER);
+            }
+            Assert.IsNull(msg);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/bin/reef-bridge-0.11.0-incubating-SNAPSHOT-shaded.jar
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/bin/reef-bridge-0.11.0-incubating-SNAPSHOT-shaded.jar b/lang/cs/Tests/ReefTests/bin/reef-bridge-0.11.0-incubating-SNAPSHOT-shaded.jar
new file mode 100644
index 0000000..e43d8bf
Binary files /dev/null and b/lang/cs/Tests/ReefTests/bin/reef-bridge-0.11.0-incubating-SNAPSHOT-shaded.jar differ

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/bin/run.cmd
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/bin/run.cmd b/lang/cs/Tests/ReefTests/bin/run.cmd
new file mode 100644
index 0000000..ddff8a7
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/bin/run.cmd
@@ -0,0 +1,45 @@
+@REM
+@REM Copyright (C) 2013 Microsoft Corporation
+@REM
+@REM Licensed under the Apache License, Version 2.0 (the "License");
+@REM you may not use this file except in compliance with the License.
+@REM You may obtain a copy of the License at
+@REM
+@REM         http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing, software
+@REM distributed under the License is distributed on an "AS IS" BASIS,
+@REM WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@REM See the License for the specific language governing permissions and
+@REM limitations under the License.
+@REM
+
+@echo off
+::
+:: Copyright (C) 2013 Microsoft Corporation
+::
+:: Licensed 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.
+::
+
+
+:: RUNTIME
+set SHADED_JAR=bin\reef-bridge-0.11.0-incubating-SNAPSHOT-shaded.jar
+
+set LOGGING_CONFIG=-Djava.util.logging.config.class=org.apache.reef.util.logging.Config
+
+set CLASSPATH=%HADOOP_HOME%\share\hadoop\hdfs\lib\*;%HADOOP_HOME%\share\hadoop\hdfs\*;%HADOOP_HOME%\share\hadoop\common\*;%HADOOP_HOME%\share\hadoop\common\lib\*;%HADOOP_HOME%\share\hadoop\mapreduce\lib\*;%HADOOP_HOME%\share\hadoop\mapreduce\*;%HADOOP_HOME%\share\hadoop\yarn\*;%HADOOP_HOME%\share\hadoop\yarn\lib\*
+
+set CMD=%JAVA_HOME%\bin\java.exe -cp %HADOOP_HOME%\etc\hadoop;%SHADED_JAR%;%CLASSPATH% %*
+::%LOGGING_CONFIG%
+echo %CMD%
+%CMD%

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/ReefTests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/ReefTests/packages.config b/lang/cs/Tests/ReefTests/packages.config
new file mode 100644
index 0000000..ef4860c
--- /dev/null
+++ b/lang/cs/Tests/ReefTests/packages.config
@@ -0,0 +1,32 @@
+<?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.Data.Edm" version="5.6.3" targetFramework="net45" />
+  <package id="Microsoft.Data.OData" version="5.6.3" targetFramework="net45" />
+  <package id="Microsoft.Data.Services.Client" version="5.6.3" targetFramework="net45" />
+  <package id="Microsoft.Hadoop.Avro" version="1.4.0.0" targetFramework="net45" />
+  <package id="Microsoft.WindowsAzure.ConfigurationManager" version="2.0.3" 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" />
+  <package id="System.Spatial" version="5.6.3" targetFramework="net45" />
+  <package id="WindowsAzure.Storage" version="4.3.0" targetFramework="net45" />
+</packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Tests/TangTests/ClassHierarchy/TestAnonymousType.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Tests/TangTests/ClassHierarchy/TestAnonymousType.cs b/lang/cs/Tests/TangTests/ClassHierarchy/TestAnonymousType.cs
new file mode 100644
index 0000000..262ea7a
--- /dev/null
+++ b/lang/cs/Tests/TangTests/ClassHierarchy/TestAnonymousType.cs
@@ -0,0 +1,63 @@
+/**
+ * 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.IO;
+using Org.Apache.Reef.Tang.Examples;
+using Org.Apache.Reef.Tang.Implementations;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Tang.Protobuf;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+
+namespace Org.Apache.Reef.Tang.Test.ClassHierarchy
+{
+    [TestClass]
+    public class TestAnonymousType
+    {
+        const string ClassHierarchyBinFileName = "example.bin";
+
+        [ClassInitialize]
+        public static void ClassSetup(TestContext context)
+        {
+            TangImpl.Reset();
+        }
+
+        [TestMethod]
+        public void TestAnonymousTypeWithDictionary()
+        {
+            List<string> appDlls = new List<string>();
+            appDlls.Add(typeof(AnonymousType).Assembly.GetName().Name);
+            var c = TangFactory.GetTang().GetClassHierarchy(appDlls.ToArray());
+            c.GetNode(typeof(AnonymousType).AssemblyQualifiedName);
+
+            IConfiguration conf = TangFactory.GetTang().NewConfigurationBuilder(c).Build();
+            IInjector injector = TangFactory.GetTang().NewInjector(conf);
+            var obj = injector.GetInstance<AnonymousType>();
+            Assert.IsNotNull(obj);
+
+            var cd = Directory.GetCurrentDirectory();
+            Console.WriteLine(cd);
+
+            ProtocolBufferClassHierarchy.Serialize(ClassHierarchyBinFileName, c);
+            IClassHierarchy ch = ProtocolBufferClassHierarchy.DeSerialize(ClassHierarchyBinFileName);
+            ch.GetNode(typeof(AnonymousType).AssemblyQualifiedName);
+        }
+    }
+}