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

[18/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-io/Network/NetworkService/NsMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-io/Network/NetworkService/NsMessage.cs b/lang/cs/Source/REEF/reef-io/Network/NetworkService/NsMessage.cs
new file mode 100644
index 0000000..46f41e0
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-io/Network/NetworkService/NsMessage.cs
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System.Collections.Generic;
+using Org.Apache.Reef.Wake;
+
+namespace Org.Apache.Reef.IO.Network.NetworkService
+{
+    /// <summary>
+    /// Message sent between NetworkServices
+    /// </summary>
+    /// <typeparam name="T">The type of data being sent</typeparam>
+    public class NsMessage<T>
+    {
+        /// <summary>
+        /// Create a new NsMessage with no data.
+        /// </summary>
+        /// <param name="sourceId">The identifier of the sender</param>
+        /// <param name="destId">The identifier of the receiver</param>
+        public NsMessage(IIdentifier sourceId, IIdentifier destId)
+        {
+            SourceId = sourceId;
+            DestId = destId;
+            Data = new List<T>();
+        }
+
+        /// <summary>
+        /// Create a new NsMessage with data.
+        /// </summary>
+        /// <param name="sourceId">The identifier of the sender</param>
+        /// <param name="destId">The identifier of the receiver</param>
+        /// <param name="message">The message to send</param>
+        public NsMessage(IIdentifier sourceId, IIdentifier destId, T message) 
+        {
+            SourceId = sourceId;
+            DestId = destId;
+            Data = new List<T> { message };
+        }
+
+        /// <summary>
+        /// The identifier of the sender of the message.
+        /// </summary>
+        public IIdentifier SourceId { get; private set; }
+
+        /// <summary>
+        /// The identifier of the receiver of the message.
+        /// </summary>
+        public IIdentifier DestId { get; private set; }
+
+        /// <summary>
+        /// A list of data being sent in the message.
+        /// </summary>
+        public List<T> Data { get; private set; } 
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-io/Network/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-io/Network/Properties/AssemblyInfo.cs b/lang/cs/Source/REEF/reef-io/Network/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..82fb3ba
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-io/Network/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("NetWork")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("NetWork")]
+[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("b3f5e608-8908-4f06-a87e-5e41c88133ac")]
+
+// 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-io/Network/Utilities/BlockingCollectionExtensions.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-io/Network/Utilities/BlockingCollectionExtensions.cs b/lang/cs/Source/REEF/reef-io/Network/Utilities/BlockingCollectionExtensions.cs
new file mode 100644
index 0000000..e9d9555
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-io/Network/Utilities/BlockingCollectionExtensions.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 System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+
+namespace Org.Apache.Reef.IO.Network.Utilities
+{
+    public static class BlockingCollectionExtensions
+    {
+        /// <summary>
+        /// Removes the given item from the BlockingCollection if it is present.
+        /// If it is not present, it blocks until any item is available in the 
+        /// BlockingCollection.  It then removes and returns that first available
+        /// item.
+        /// </summary>
+        /// <typeparam name="T">The type of BlockingCollection</typeparam>
+        /// <param name="collection">The BlockingCollection to remove the specified item</param>
+        /// <param name="item">The item to remove from the BlockingCollection, if it exists</param>
+        /// <returns>The specified item, or the first available item if the specified item is 
+        /// not present in the BlockingCollection</returns>
+        public static T Take<T>(this BlockingCollection<T> collection, T item)
+        {
+            T ret = default(T);
+            bool foundItem = false; 
+            List<T> removedItems = new List<T>();
+
+            // Empty the collection
+            for (int i = 0; i < collection.Count; i++)
+            {
+                T removed;
+                if (collection.TryTake(out removed))
+                {
+                    removedItems.Add(removed);
+                }
+            }
+
+            // Add them back to the collection minus the specified item
+            foreach (T removedItem in removedItems)
+            {
+                if (removedItem.Equals(item))
+                {
+                    ret = removedItem;
+                    foundItem = true;
+                }
+                else
+                {
+                    collection.Add(removedItem);
+                }
+            }
+
+            if (!foundItem)
+            {
+                // Error: the element wasn't in the collection
+                throw new InvalidOperationException(item + " not found in blocking collection");
+            }
+
+            return ret;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-io/Network/Utilities/Utils.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-io/Network/Utilities/Utils.cs b/lang/cs/Source/REEF/reef-io/Network/Utilities/Utils.cs
new file mode 100644
index 0000000..91bb90e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-io/Network/Utilities/Utils.cs
@@ -0,0 +1,76 @@
+/**
+ * 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.IO;
+using Microsoft.Hadoop.Avro;
+using Org.Apache.Reef.Driver.Context;
+using Org.Apache.Reef.Tasks;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Exceptions;
+using Org.Apache.Reef.Tang.Implementations;
+using Org.Apache.Reef.Tang.Interface;
+using Org.Apache.Reef.Tang.Util;
+
+namespace Org.Apache.Reef.IO.Network.Utilities
+{
+    internal class Utils
+    {
+        private static Logger LOGGER = Logger.GetLogger(typeof(Utils));
+
+        /// <summary>
+        /// Returns the TaskIdentifier from the Configuration.
+        /// </summary>
+        /// <param name="taskConfiguration">The Configuration object</param>
+        /// <returns>The TaskIdentifier for the given Configuration</returns>
+        public static string GetTaskId(IConfiguration taskConfiguration)
+        {
+            try
+            {
+                IInjector injector = TangFactory.GetTang().NewInjector(taskConfiguration);
+                return injector.GetNamedInstance<TaskConfigurationOptions.Identifier, string>(
+                    GenericType<TaskConfigurationOptions.Identifier>.Class);
+            }
+            catch (InjectionException)
+            {
+                LOGGER.Log(Level.Error, "Unable to find task identifier");
+                throw;
+            }
+        }
+
+        /// <summary>
+        /// Returns the Context Identifier from the Configuration.
+        /// </summary>
+        /// <param name="contextConfiguration">The Configuration object</param>
+        /// <returns>The TaskIdentifier for the given Configuration</returns>
+        public static string GetContextId(IConfiguration contextConfiguration)
+        {
+            try
+            {
+                IInjector injector = TangFactory.GetTang().NewInjector(contextConfiguration);
+                return injector.GetNamedInstance<ContextConfigurationOptions.ContextIdentifier, string>(
+                    GenericType<ContextConfigurationOptions.ContextIdentifier>.Class);
+            }
+            catch (InjectionException)
+            {
+                LOGGER.Log(Level.Error, "Unable to find task identifier");
+                throw;
+            }
+        }
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/FailedTask/FailedTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/FailedTask/FailedTask.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/FailedTask/FailedTask.cs
new file mode 100644
index 0000000..64e1d96
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/FailedTask/FailedTask.cs
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Threading;
+
+namespace Org.Apache.Reef.Tasks
+{
+    public class FailedTask : ITask
+    {
+        [Inject]
+        public FailedTask()
+        {
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            Console.WriteLine("I am about to fail.");
+            Thread.Sleep(2 * 1000);
+            throw new ApplicationException("bite me.");
+        }
+
+        public void Dispose()
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloService.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloService.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloService.cs
new file mode 100644
index 0000000..ee3beeb
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloService.cs
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Services;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Collections.Generic;
+
+namespace Org.Apache.Reef.Tasks
+{
+    public class HelloService : IService
+    {
+        private IList<string> _guests;
+
+        [Inject]
+        public HelloService()
+        {
+            if (_guests == null)
+            {
+                _guests = new List<string>();
+                _guests.Add("MR.SMITH");
+            }
+        }
+
+        public IList<string> Guests
+        {
+            get
+            {
+                return _guests;
+            }
+        }
+
+        public void AddGuest(string guestName)
+        {
+            if (string.IsNullOrWhiteSpace(guestName))
+            {
+                throw new ArgumentException("can't do with empty name.");
+            }
+            Guests.Add(guestName);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTask.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTask.cs
new file mode 100644
index 0000000..f1a7328
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTask.cs
@@ -0,0 +1,123 @@
+/**
+ * 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.Tasks.Events;
+using Org.Apache.Reef.Utilities;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Linq;
+using System.Net;
+using System.Threading;
+
+namespace Org.Apache.Reef.Tasks
+{
+    public class HelloTask : ITask
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(HelloTask));
+
+        private INameClient _nameClient = null;
+
+        [Inject]
+        public HelloTask()
+        {
+            Console.WriteLine("HelloTask constructor 0");
+        }
+
+        [Inject]
+        public HelloTask(HelloService service, INameClient nameClient)
+        {
+            Console.WriteLine("HelloTask constructor 2");
+            Service = service;
+            _nameClient = nameClient;
+        }
+
+        [Inject]
+        public HelloTask(HelloService service)
+        {
+            Console.WriteLine("HelloTask constructor 1");
+            Service = service;
+        }
+
+        public HelloService Service { get; set; }
+
+        public byte[] Call(byte[] memento)
+        {
+            Console.WriteLine("Hello, CLR REEF!");
+            if (_nameClient != null)
+            {
+                _nameClient.Register("abc", new IPEndPoint(IPAddress.Any, 8080));
+                Console.WriteLine("IP Address: {0}", _nameClient.Lookup("abc"));
+            }
+            PrintGuestList();
+            Thread.Sleep(5 * 1000);
+            Console.WriteLine("Bye, CLR REEF!");
+
+            return null;
+        }
+
+        public void Dispose()
+        {
+            LOGGER.Log(Level.Info, "Hello task disposed.");
+        }
+
+        private void HandleDriverMessage(string message)
+        {
+            using (LOGGER.LogFunction("HelloTask::HandleDriverMessage"))
+            {
+                LOGGER.Log(Level.Info, "I handle message by logging : " + message);
+            }
+        }
+
+        private void PrintGuestList()
+        {
+            if (Service == null || !Service.Guests.Any())
+            {
+                Console.WriteLine("No service provided.");
+            }
+            else
+            {
+                Console.WriteLine("Serving guest: " + string.Join(";", Service.Guests));
+            }
+        }
+
+        public class HelloDriverMessageHandler : IDriverMessageHandler
+        {
+            private HelloTask _parentTask;
+
+            [Inject]
+            public HelloDriverMessageHandler(HelloTask task)
+            {
+                _parentTask = task;
+            }
+
+            public void Handle(IDriverMessage value)
+            {
+                string message = string.Empty;
+                LOGGER.Log(Level.Verbose, "Receieved a message from driver, handling it with HelloDriverMessageHandler");
+                if (value.Message.IsPresent())
+                {
+                    message = ByteUtilities.ByteArrarysToString(value.Message.Value);
+                }
+                _parentTask.HandleDriverMessage(message);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTaskMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTaskMessage.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTaskMessage.cs
new file mode 100644
index 0000000..b04043e
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/HelloTask/HelloTaskMessage.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.Utilities;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Globalization;
+
+namespace Org.Apache.Reef.Tasks
+{
+    public class HelloTaskMessage : ITaskMessageSource
+    {
+        [Inject]
+        public HelloTaskMessage()
+        {
+        }
+
+        public Optional<TaskMessage> Message
+        {
+            get
+            {
+                TaskMessage defaultTaskMessage = TaskMessage.From(
+                    "helloSourceId",
+                    ByteUtilities.StringToByteArrays("hello message generated at " + DateTime.Now.ToString(CultureInfo.InvariantCulture)));
+                return Optional<TaskMessage>.Of(defaultTaskMessage);
+            }
+
+            set
+            {
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/Properties/AssemblyInfo.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..4a41805
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/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("Tasks")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("Tasks")]
+[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("b9e219f1-a02c-468c-ab26-3ef5c91310f7")]
+
+// 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-tasks/Tasks/ShellTask/ShellTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/ShellTask/ShellTask.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/ShellTask/ShellTask.cs
new file mode 100644
index 0000000..07ab0b2
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/ShellTask/ShellTask.cs
@@ -0,0 +1,90 @@
+/**
+ * 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.Diagnostics;
+using Org.Apache.Reef.Utilities.Logging;
+using Org.Apache.Reef.Tang.Annotations;
+using System;
+using System.Diagnostics;
+using System.Diagnostics.CodeAnalysis;
+using System.Globalization;
+using System.Text;
+
+[module: SuppressMessage("StyleCop.CSharp.MaintainabilityRules", "SA1402:FileMayOnlyContainASingleClass", Justification = "allow name parameter class to be embedded")]
+
+namespace Org.Apache.Reef.Tasks
+{
+    public class ShellTask : ITask
+    {
+        [Inject]
+        public ShellTask([Parameter(Value = typeof(Command))] string command)
+        {
+            Cmd = command;
+        }
+
+        public string Cmd { get; set; }
+
+        public byte[] Call(byte[] memento)
+        {
+            return Encoding.UTF8.GetBytes(CommandLineExecute(Cmd));
+        }
+
+        public void Dispose()
+        {
+        }
+
+        private string CommandLineExecute(string command)
+        {
+            string output = string.Empty;
+            try
+            {
+                ProcessStartInfo startInfo = new ProcessStartInfo()
+                {
+                    FileName = "cmd.exe",
+                    Arguments = @"/c " + command,
+                    RedirectStandardOutput = true,
+                    UseShellExecute = false,
+                    CreateNoWindow = true
+                };
+
+                using (Process process = Process.Start(startInfo))
+                {
+                    StringBuilder standardOutput = new StringBuilder();
+
+                    process.WaitForExit(1000);
+
+                    standardOutput.Append(process.StandardOutput.ReadToEnd());
+                    output = standardOutput.ToString();
+                }
+            }
+            catch (Exception e)
+            {
+                output = string.Format(CultureInfo.InvariantCulture, "Failed to execute command [{0}] and capture the output, exception {1} with message {2} ", command, e, e.Message);
+                Exceptions.Caught(e, Level.Error, output, Logger.GetLogger(typeof(ShellTask)));
+            }
+
+            return output;
+        }
+
+        [NamedParameter("Shell Command", "cmd", "")]
+        public class Command : Name<string>
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask1.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask1.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask1.cs
new file mode 100644
index 0000000..8e05a76
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask1.cs
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System.Threading;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tasks
+{
+    public class StreamTask1 : ITask
+    {
+        private string _ipAddress;
+
+        [Inject]
+        public StreamTask1([Parameter(typeof(IpAddress))] string ipAddress)
+        {
+            this._ipAddress = ipAddress;
+        }
+
+        [Inject]
+        public StreamTask1()
+        {
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            System.Console.WriteLine("Hello, Streaming 1!!, Ip: " + _ipAddress);
+
+            Thread.Sleep(10000);
+
+            SIFirstNode();
+
+            return null;
+        }
+
+        public void Dispose()
+        {
+        }
+
+        public void SIFirstNode()
+        {
+            //var a = new SIFirstNodeXAM();
+            //a.Process(1111, 2222, ipAddress);
+        }
+
+        [NamedParameter("Ip Address", "IP", "10.121.32.158")]
+        public class IpAddress : Name<string>
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask2.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask2.cs b/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask2.cs
new file mode 100644
index 0000000..7c521f1
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/StreamingTasks/StreamTask2.cs
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tasks
+{
+    public class StreamTask2 : ITask
+    {
+        [Inject]
+        public StreamTask2()
+        {
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            System.Console.WriteLine("Hello, Streaming 2!!");
+
+            SISecondNode();
+
+            return null;
+        }
+
+        public void Dispose()
+        {
+        }
+
+        public void SISecondNode()
+        {
+            //var a = new SISecondNodeXAM();
+            //a.Process(2222, 1111);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/REEF/reef-tasks/Tasks/Tasks.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Source/REEF/reef-tasks/Tasks/Tasks.csproj b/lang/cs/Source/REEF/reef-tasks/Tasks/Tasks.csproj
new file mode 100644
index 0000000..8cae084
--- /dev/null
+++ b/lang/cs/Source/REEF/reef-tasks/Tasks/Tasks.csproj
@@ -0,0 +1,93 @@
+<?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>{75503F90-7B82-4762-9997-94B5C68F15DB}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Org.Apache.Reef.Tasks</RootNamespace>
+    <AssemblyName>Org.Apache.Reef.Tasks</AssemblyName>
+    <TargetFrameworkVersion>v4.5</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+  </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.Tasks\</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.Tasks\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="System" />
+    <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="FailedTask\FailedTask.cs" />
+    <Compile Include="HelloTask\HelloService.cs" />
+    <Compile Include="HelloTask\HelloTask.cs" />
+    <Compile Include="HelloTask\HelloTaskMessage.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="ShellTask\ShellTask.cs" />
+    <Compile Include="StreamingTasks\StreamTask1.cs" />
+    <Compile Include="StreamingTasks\StreamTask2.cs" />
+  </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="..\..\reef-common\ReefCommon\ReefCommon.csproj">
+      <Project>{545a0582-4105-44ce-b99c-b1379514a630}</Project>
+      <Name>ReefCommon</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.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/TANG/Examples/AnonymousType.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/AnonymousType.cs b/lang/cs/Source/TANG/Examples/AnonymousType.cs
new file mode 100644
index 0000000..9124409
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/AnonymousType.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 System;
+using System.Collections.Generic;
+using System.Linq;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class MyEventStreamDefinition
+    {
+        public ISet<int> Shards { get; set; }
+
+        public Type Type { get; set; }
+    }
+
+    public class AnonymousType
+    {
+        private Dictionary<string, MyEventStreamDefinition> d = new Dictionary<string, MyEventStreamDefinition>();
+        private Dictionary<string, int> d2;
+
+        //Anonymous class in injectable constructor
+        [Inject]
+        public AnonymousType()
+        {
+            d2 = d
+                .Select((e, i) => new { e.Key, i })
+                .ToDictionary(e => e.Key, e => e.i);
+        }
+
+        //Anonymous class in other constructor
+        public AnonymousType(Dictionary<string, MyEventStreamDefinition> d)
+        {
+            this.d = d;            
+            d2 = d
+                .Select((e, i) => new { e.Key, i })
+                .ToDictionary(e => e.Key, e => e.i);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/CheckChild.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/CheckChild.cs b/lang/cs/Source/TANG/Examples/CheckChild.cs
new file mode 100644
index 0000000..b4415cb
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/CheckChild.cs
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public interface CheckChildIface 
+    {
+    }   
+    
+    public class CheckChildImpl : CheckChildIface
+    {
+        [Inject]
+        public CheckChildImpl() {}
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/DocumentedLocalNamedParameter.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/DocumentedLocalNamedParameter.cs b/lang/cs/Source/TANG/Examples/DocumentedLocalNamedParameter.cs
new file mode 100644
index 0000000..a23a424
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/DocumentedLocalNamedParameter.cs
@@ -0,0 +1,43 @@
+/**
+ * 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.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class DocumentedLocalNamedParameter
+    {
+        string value;
+        [NamedParameter(Documentation = "doc stuff", ShortName = "DocFoo", DefaultValue = "some value")]
+        public sealed class Foo : Name<String> 
+        {
+        }
+
+        [Inject]
+        public DocumentedLocalNamedParameter([Parameter(typeof(Foo))] String s) 
+        {
+            value = s;
+        }
+
+        public override string ToString()
+        {
+            return value;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/Examples.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/Examples.csproj b/lang/cs/Source/TANG/Examples/Examples.csproj
new file mode 100644
index 0000000..ac5a7e3
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/Examples.csproj
@@ -0,0 +1,95 @@
+<?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>{31B4389E-925A-4181-A1F6-21A1A0AD8A1C}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Org.Apache.Reef.Tang.Examples</RootNamespace>
+    <AssemblyName>Org.Apache.Reef.Tang.Examples</AssemblyName>
+    <TargetFrameworkVersion>v4.5</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+  </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.Tang.Examples\</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.Tang.Examples\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="System" />
+    <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="AnonymousType.cs" />
+    <Compile Include="CheckChild.cs" />
+    <Compile Include="DocumentedLocalNamedParameter.cs" />
+    <Compile Include="FileNames.cs" />
+    <Compile Include="ForksInjectorInConstructor.cs" />
+    <Compile Include="GenericArgument.cs" />
+    <Compile Include="GenericAsArgument.cs" />
+    <Compile Include="NamedParameterConstructors.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="ShortNameFooAB.cs" />
+    <Compile Include="SimpleConstructors.cs" />
+    <Compile Include="TestClasses.cs" />
+    <Compile Include="Timer.cs" />
+    <Compile Include="TweetExample.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\..\WAKE\Wake\Wake.csproj">
+      <Project>{cdfb3464-4041-42b1-9271-83af24cd5008}</Project>
+      <Name>Wake</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\Tang\Tang.csproj">
+      <Project>{97dbb573-3994-417a-9f69-ffa25f00d2a6}</Project>
+      <Name>Tang</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.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/TANG/Examples/FileNames.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/FileNames.cs b/lang/cs/Source/TANG/Examples/FileNames.cs
new file mode 100644
index 0000000..d371908
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/FileNames.cs
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class FileNames
+    {
+        public static string Examples = @"Org.Apache.Reef.Tang.Examples";
+        public static string Common = @"Org.Apache.Reef.Common";
+        public static string Tasks = @"Org.Apache.Reef.Tasks";
+        public static string Seconds = "Org.Apache.Reef.Tang.Examples.Timer+Seconds";
+        public static string Timer = "Org.Apache.Reef.Tang.Examples.Timer";
+        public static string B = "Org.Apache.Reef.Tang.Examples.B";
+        public static string B1 = "Org.Apache.Reef.Tang.Examples.B+B1";
+        public static string B2 = "Org.Apache.Reef.Tang.Examples.B+B1+B2";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/ForksInjectorInConstructor.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/ForksInjectorInConstructor.cs b/lang/cs/Source/TANG/Examples/ForksInjectorInConstructor.cs
new file mode 100644
index 0000000..c78141a
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/ForksInjectorInConstructor.cs
@@ -0,0 +1,59 @@
+/**
+ * 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;
+/**
+ * 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.Linq;
+using System.Text;
+using System.Threading.Tasks;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Tang.Implementations;
+using Org.Apache.Reef.Tang.Interface;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class ForksInjectorInConstructor
+    {
+        [Inject]
+        public ForksInjectorInConstructor(IInjector i)
+        {
+            ICsConfigurationBuilder cb = TangFactory.GetTang().NewConfigurationBuilder(new string[] { @"Org.Apache.Reef.Tang.Examples" });
+            //cb.BindImplementation(Number.class, typeof(Int32));
+            i.ForkInjector(new IConfiguration[] { cb.Build() });
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/GenericArgument.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/GenericArgument.cs b/lang/cs/Source/TANG/Examples/GenericArgument.cs
new file mode 100644
index 0000000..dd1a228
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/GenericArgument.cs
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using Org.Apache.Reef.Tang.Annotations;
+using Org.Apache.Reef.Wake.RX;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class GenericArgument<T> : AbstractObserver<T>
+    {
+        [Inject]
+        public GenericArgument()
+        {        
+        }
+
+        public override void OnNext(T arg1)
+        {
+            throw new NotImplementedException();
+        }
+    }
+
+    public class ClassWithGenericArgument<T>
+    {
+        private T _obj;
+
+        public ClassWithGenericArgument(T obj)
+        {
+            _obj = obj;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/GenericAsArgument.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/GenericAsArgument.cs b/lang/cs/Source/TANG/Examples/GenericAsArgument.cs
new file mode 100644
index 0000000..54e73ab
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/GenericAsArgument.cs
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class Event<TPayload>
+    {
+        public Event()
+        {
+        }
+
+        public Event(StreamEvent<TPayload> trillEvent)
+        {
+        }
+
+        public StreamEvent<TPayload> ToStreamEvent()
+        {
+            return null;
+        }
+    }
+
+    public class StreamEvent<T>
+    {
+        public StreamEvent()
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/NamedParameterConstructors.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/NamedParameterConstructors.cs b/lang/cs/Source/TANG/Examples/NamedParameterConstructors.cs
new file mode 100644
index 0000000..347c6a4
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/NamedParameterConstructors.cs
@@ -0,0 +1,40 @@
+/**
+ * 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.Text;
+using System.Threading.Tasks;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class NamedParameterConstructors
+    {
+        [NamedParameter()]
+        public class X : Name<String> 
+        {
+        };
+
+        [Inject]
+        public NamedParameterConstructors(String x, [Parameter(typeof(X))] String y) 
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/Properties/AssemblyInfo.cs b/lang/cs/Source/TANG/Examples/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..dcb9bc5
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/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("Examples")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("Examples")]
+[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("cb44520c-2fd1-4d3b-9b9a-250f94a11239")]
+
+// 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/TANG/Examples/ShortNameFooAB.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/ShortNameFooAB.cs b/lang/cs/Source/TANG/Examples/ShortNameFooAB.cs
new file mode 100644
index 0000000..ae81a82
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/ShortNameFooAB.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.
+ */
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    [NamedParameter(ShortName = "fooA")]
+    public class ShortNameFooA : Name<String> 
+    {
+    }
+
+    //when same short name is used, exception would throw when building the class hierarchy
+    [NamedParameter(ShortName = "fooB")]
+    public class ShortNameFooB : Name<Int32> 
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/SimpleConstructors.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/SimpleConstructors.cs b/lang/cs/Source/TANG/Examples/SimpleConstructors.cs
new file mode 100644
index 0000000..2a4aa4c
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/SimpleConstructors.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 System;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class SimpleConstructors 
+    {
+        [Inject]
+        public SimpleConstructors() 
+        {
+        }
+
+        [Inject]
+        public SimpleConstructors(int x) 
+        {
+        }
+
+        [Inject]
+        public SimpleConstructors(int x, String y) 
+        {
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/TestClasses.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/TestClasses.cs b/lang/cs/Source/TANG/Examples/TestClasses.cs
new file mode 100644
index 0000000..03a7c70
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/TestClasses.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 System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public interface A
+    {
+
+    }
+
+    public class B : A
+    {
+        public class B1
+        {
+            public class B2 { }
+        }
+    }
+
+    public class C : B
+    {
+        string s;
+        int v;
+
+        [Inject]
+        public C(string s, int v)
+        {
+            this.s = s;
+            this.v = v;
+        }
+    }
+
+    public static class E
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/Timer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/Timer.cs b/lang/cs/Source/TANG/Examples/Timer.cs
new file mode 100644
index 0000000..a65d717
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/Timer.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 System;
+using System.Threading;
+using Org.Apache.Reef.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public class Timer
+    {
+        [NamedParameter("Number of seconds to sleep", "sec", "10")]
+        public class Seconds : Name<Int32> { }
+        private readonly int seconds;
+
+        [Inject]
+        public Timer([Parameter(typeof(Seconds))] int seconds)
+        {
+            if (seconds < 0)
+            {
+                throw new ArgumentException("Cannot sleep for negative time!");
+            }
+            this.seconds = seconds;
+        }
+
+        public void sleep()  
+        {
+            Thread.Sleep(seconds * 1000);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Examples/TweetExample.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Examples/TweetExample.cs b/lang/cs/Source/TANG/Examples/TweetExample.cs
new file mode 100644
index 0000000..774f48d
--- /dev/null
+++ b/lang/cs/Source/TANG/Examples/TweetExample.cs
@@ -0,0 +1,90 @@
+/**
+ * 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.Tang.Annotations;
+
+namespace Org.Apache.Reef.Tang.Examples
+{
+    public interface ISMS
+    {
+        void SendSMS(String msg, long phoneNumber);
+    }
+
+    public interface ITweetFactory
+    {
+        String GetTweet();
+    } 
+
+    public class MockTweetFactory : ITweetFactory 
+    {
+        [Inject]
+        public MockTweetFactory() {
+        }
+
+        public String GetTweet() 
+        {
+            return "@tw #bbq bbqftw!!! gopher://vuwed.wefd/bbqftw!";
+        }
+    }
+
+    public class MockSMS : ISMS 
+    {
+        [Inject]
+        public MockSMS() 
+        {
+        }
+
+        public void SendSMS(String msg, long phoneNumber) 
+        {
+            if (phoneNumber != 8675309) 
+            {
+                throw new ArgumentException("Unknown recipient");
+            }
+            // success!
+        }
+    }
+
+    public class Tweeter
+    {
+        ITweetFactory tw;
+        ISMS sms;
+        long phoneNumber;
+
+        [NamedParameter(Documentation="Phone number", ShortName="number", DefaultValue="1800")]
+        public class PhoneNumber : Name<long> { }
+        [Inject]
+        public Tweeter(ITweetFactory tw, ISMS sms, [Parameter(typeof(PhoneNumber))] long phoneNumber)
+        {
+            this.tw = tw;
+            this.sms = sms;
+            this.phoneNumber = phoneNumber;
+        }
+
+        [Inject]
+        public Tweeter([Parameter(typeof(PhoneNumber))] long phoneNumber)
+        {
+            this.phoneNumber = phoneNumber;
+        }
+
+        public void sendMessage()
+        {
+            sms.SendSMS(tw.GetTweet(), phoneNumber);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Tang/Annotations/DefaultImplementation.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Tang/Annotations/DefaultImplementation.cs b/lang/cs/Source/TANG/Tang/Annotations/DefaultImplementation.cs
new file mode 100644
index 0000000..26ac15b
--- /dev/null
+++ b/lang/cs/Source/TANG/Tang/Annotations/DefaultImplementation.cs
@@ -0,0 +1,43 @@
+/**
+ * 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.Text;
+using System.Threading.Tasks;
+
+namespace Org.Apache.Reef.Tang.Annotations
+{
+    /// <summary>
+    /// DefaultImplementationAttribute
+    /// </summary>
+    [AttributeUsage(AttributeTargets.Class | AttributeTargets.Interface, Inherited = false)]
+    public class DefaultImplementationAttribute : System.Attribute
+    {
+        public Type Value { get; set; }
+        public string Name { get; set; }
+
+        public DefaultImplementationAttribute(Type val = null, string n = "")
+        {
+            Name = n;
+            Value = val;
+        }
+
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Tang/Annotations/Inject.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Tang/Annotations/Inject.cs b/lang/cs/Source/TANG/Tang/Annotations/Inject.cs
new file mode 100644
index 0000000..2816a18
--- /dev/null
+++ b/lang/cs/Source/TANG/Tang/Annotations/Inject.cs
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+namespace Org.Apache.Reef.Tang.Annotations
+{
+    /// <summary>
+    /// InjectAttribute
+    /// </summary>
+    [System.AttributeUsage(System.AttributeTargets.Constructor)]
+    public class InjectAttribute : System.Attribute 
+    { 
+    }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Tang/Annotations/NamedParameter.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Tang/Annotations/NamedParameter.cs b/lang/cs/Source/TANG/Tang/Annotations/NamedParameter.cs
new file mode 100644
index 0000000..d156a32
--- /dev/null
+++ b/lang/cs/Source/TANG/Tang/Annotations/NamedParameter.cs
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+using System;
+
+namespace Org.Apache.Reef.Tang.Annotations
+{
+    [System.AttributeUsage(System.AttributeTargets.Class)]
+    public class NamedParameterAttribute : System.Attribute
+    {
+        public string Documentation { get; set; }
+        public string ShortName { get; set; }
+        public string DefaultValue { get; set; }
+        public Type DefaultClass { get; set; }
+        public string[] DefaultValues { get; set; }
+        public Type[] DefaultClasses { get; set; }
+
+        public NamedParameterAttribute(string documentation = "", string shortName = "",
+            string defaultValue = "", Type defaultClass = null, string[] defaultValues = null, Type[] defaultClasses = null)
+        {
+            this.Documentation = documentation;
+            this.ShortName = shortName;
+            this.DefaultValue = defaultValue;
+            this.DefaultClass = defaultClass;
+            this.DefaultValues = defaultValues;
+            this.DefaultClasses = defaultClasses;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Tang/Annotations/Parameter.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Tang/Annotations/Parameter.cs b/lang/cs/Source/TANG/Tang/Annotations/Parameter.cs
new file mode 100644
index 0000000..7c3c2ae
--- /dev/null
+++ b/lang/cs/Source/TANG/Tang/Annotations/Parameter.cs
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+using System;
+namespace Org.Apache.Reef.Tang.Annotations
+{
+    /// <summary>
+    /// ParameterAttribute
+    /// </summary>
+    [System.AttributeUsage(System.AttributeTargets.Parameter)]
+    public class ParameterAttribute : System.Attribute
+    {
+        public ParameterAttribute()
+        {
+        }
+
+        public ParameterAttribute(Type value)
+        {
+            Value = value;
+        }
+
+        public Type Value { get; set; } 
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Tang/Annotations/Unit.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Tang/Annotations/Unit.cs b/lang/cs/Source/TANG/Tang/Annotations/Unit.cs
new file mode 100644
index 0000000..bd3cad0
--- /dev/null
+++ b/lang/cs/Source/TANG/Tang/Annotations/Unit.cs
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+using System;
+namespace Org.Apache.Reef.Tang.Annotations
+{
+    /// <summary>
+    /// UnitAttribute
+    /// </summary>
+    [AttributeUsage(AttributeTargets.Class, Inherited = false)]
+    public class UnitAttribute : System.Attribute 
+    { 
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Tang/Exceptions/BindException.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Tang/Exceptions/BindException.cs b/lang/cs/Source/TANG/Tang/Exceptions/BindException.cs
new file mode 100644
index 0000000..2ad1658
--- /dev/null
+++ b/lang/cs/Source/TANG/Tang/Exceptions/BindException.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;
+
+namespace Org.Apache.Reef.Tang.Exceptions
+{
+    public class BindException : SystemException
+    {
+        //private static readonly long serialVersionUID = 1L;
+        public BindException(String message)
+            : base(message)
+        {           
+        }
+
+        public BindException(string message, Exception innerException)
+            : base(message, innerException)
+        {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/2ae282de/lang/cs/Source/TANG/Tang/Exceptions/ClassHierarchyException.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Source/TANG/Tang/Exceptions/ClassHierarchyException.cs b/lang/cs/Source/TANG/Tang/Exceptions/ClassHierarchyException.cs
new file mode 100644
index 0000000..5b29f35
--- /dev/null
+++ b/lang/cs/Source/TANG/Tang/Exceptions/ClassHierarchyException.cs
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+using System;
+
+namespace Org.Apache.Reef.Tang.Exceptions
+{
+    public class ClassHierarchyException : SystemException
+    {
+        public ClassHierarchyException(String msg) :  base(msg)
+        {           
+        }
+
+        public ClassHierarchyException(string message, Exception innerException)
+            : base(message, innerException)
+        {
+        }
+    }
+}
\ No newline at end of file