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/06/30 22:29:11 UTC

incubator-reef git commit: [REEF-396] Implement ClassHierarchySerializer using Avro on the .Net side This PR includes

Repository: incubator-reef
Updated Branches:
  refs/heads/master 2a37f80d4 -> e7550466d


[REEF-396]  Implement ClassHierarchySerializer using Avro on the .Net side
This PR includes

  * AvroClassHierarchy
  * AvroClassHierarchySerializer
  * Avro data contract for ClassHierarchy
  * Test cases for the conversion between AvroNode and ClassHierarchy

There are some issues in Avro library for serialization. That portion
will be tested after the bug from Avro team is resolved.

JIRA:
  [REEF-396](https://issues.apache.org/jira/browse/REEF-396)

Pull Request:
  This closes #253


Project: http://git-wip-us.apache.org/repos/asf/incubator-reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-reef/commit/e7550466
Tree: http://git-wip-us.apache.org/repos/asf/incubator-reef/tree/e7550466
Diff: http://git-wip-us.apache.org/repos/asf/incubator-reef/diff/e7550466

Branch: refs/heads/master
Commit: e7550466d144f95bf9aa74b8cdf7caf767c7a402
Parents: 2a37f80
Author: Julia Wang <jw...@yahoo.com>
Authored: Fri Jun 26 18:36:56 2015 -0700
Committer: Markus Weimer <we...@apache.org>
Committed: Tue Jun 30 13:23:48 2015 -0700

----------------------------------------------------------------------
 .../ClassHierarchy/TestAvroSerialization.cs     | 278 +++++++++++
 .../Org.Apache.REEF.Tang.Tests.csproj           |   4 +
 .../ClassHierarchy/AvroClassHierarchy.cs        | 291 +++++++++++
 .../AvroClassHierarchySerializer.cs             | 483 +++++++++++++++++++
 .../AvroDataContract/AvroClassNode.cs           | 120 +++++
 .../AvroDataContract/AvroConstructorArg.cs      |  86 ++++
 .../AvroDataContract/AvroConstructorDef.cs      |  78 +++
 .../AvroDataContract/AvroNamedParameterNode.cs  | 120 +++++
 .../ClassHierarchy/AvroDataContract/AvroNode.cs | 116 +++++
 .../AvroDataContract/AvroPackageNode.cs         |  54 +++
 .../Interface/IClassHierarchySerializer.cs      |  84 ++++
 .../Org.Apache.REEF.Tang.csproj                 |   9 +
 12 files changed, 1723 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang.Tests/ClassHierarchy/TestAvroSerialization.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Tests/ClassHierarchy/TestAvroSerialization.cs b/lang/cs/Org.Apache.REEF.Tang.Tests/ClassHierarchy/TestAvroSerialization.cs
new file mode 100644
index 0000000..b671392
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang.Tests/ClassHierarchy/TestAvroSerialization.cs
@@ -0,0 +1,278 @@
+/**
+ * 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.IO;
+using Microsoft.Hadoop.Avro;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Org.Apache.REEF.Examples.Tasks.HelloTask;
+using Org.Apache.REEF.Tang.Examples;
+using Org.Apache.REEF.Tang.Implementations.ClassHierarchy;
+using Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Types;
+
+namespace Org.Apache.REEF.Tang.Tests.ClassHierarchy
+{
+    [TestClass]
+    public class TestAvroSerialization
+    {
+        private AvroClassHierarchySerializer _serializer =
+            (AvroClassHierarchySerializer)TangFactory.GetTang().NewInjector().GetInstance<IClassHierarchySerializer>();
+
+        [ClassInitialize]
+        public static void ClassSetup(TestContext context)
+        {
+        }
+
+        [ClassCleanup]
+        public static void ClassCleanup()
+        {
+        }
+
+        [TestInitialize()]
+        public void TestSetup()
+        {
+        }
+
+        [TestCleanup()]
+        public void TestCleanup()
+        {
+        }
+
+        /// <summary>
+        /// This test is to convert an ClassHierarchy into AvroNode object, and then convert it back to an 
+        /// AvroClassHierarchy. 
+        /// </summary>
+        [TestMethod]
+        public void TestToFromAvroNode()
+        {
+            Type timerType = typeof(Timer);
+            Type secondType = typeof(Timer.Seconds);
+            Type simpleCOnstuctorType = typeof(SimpleConstructors);
+
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(
+                new string[] { typeof(Timer).Assembly.GetName().Name, typeof(SimpleConstructors).Assembly.GetName().Name });
+            IClassNode timerClassNode = (IClassNode)ns.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode = ns.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode = (IClassNode)ns.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            AvroNode n =_serializer.ToAvroNode(ns);
+            IClassHierarchy ns2 = _serializer.FromAvroNode(n);
+
+            IClassNode timerClassNode2 = (IClassNode)ns2.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode2 = ns2.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode2 = (IClassNode)ns2.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            Assert.AreEqual(timerClassNode, timerClassNode2);
+            Assert.AreEqual(secondNode, secondNode2);
+            Assert.AreEqual(simpleConstructorsClassNode, simpleConstructorsClassNode2);
+        }
+
+        /// <summary>
+        /// This is to test AvroSerializer.Create<AvroNode>() 
+        /// Manually changing return type from object into strong type like AvroClassNode in AvroNode class will result in GetSchema() fail. 
+        /// I have manully synced all the IList into List in auto generated code. Otherswise AvroSerializer.Create<AvroNode>() will throw the following error
+        /// "Could not find any matching known type for 'System.Collections.Generic.IList`1[Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorDef]'."
+        /// </summary>
+        [TestMethod]
+        public void TestGetSchema()
+        {
+            var serializer = AvroSerializer.Create<AvroNode>();
+            var s =  serializer.WriterSchema.ToString();
+            Assert.IsNotNull(s);
+        }
+
+        /// <summary>
+        /// This test is to merge two AvroClassHierarchies
+        /// </summary>
+        [TestMethod]
+        public void TestAvroClassHierarchyMerge()
+        {
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(
+                new string[] { typeof(Timer).Assembly.GetName().Name });
+            IClassNode timerClassNode = (IClassNode)ns.GetNode(typeof(Timer).AssemblyQualifiedName);
+
+            AvroNode n = _serializer.ToAvroNode(ns);
+            IClassHierarchy ns2 = _serializer.FromAvroNode(n);
+
+            IClassHierarchy ns3 = TangFactory.GetTang().GetClassHierarchy(
+                    new string[] { typeof(AvroNode).Assembly.GetName().Name });
+            IClassNode avroNodeClassNode = (IClassNode)ns3.GetNode(typeof(AvroNode).AssemblyQualifiedName);
+
+            AvroNode n2 = _serializer.ToAvroNode(ns3);
+            IClassHierarchy ns4 = _serializer.FromAvroNode(n2);
+
+            var ns5 = ns2.Merge(ns4);
+
+            IClassNode timerClassNode2 = (IClassNode)ns5.GetNode(typeof(Timer).AssemblyQualifiedName);
+            IClassNode avroNodeClassNode2 = (IClassNode)ns5.GetNode(typeof(AvroNode).AssemblyQualifiedName);
+
+            Assert.AreEqual(timerClassNode, timerClassNode2);
+            Assert.AreEqual(avroNodeClassNode, avroNodeClassNode2);
+        }
+
+        /// <summary>
+        /// Test serialize a class hierarchy into a test file
+        /// </summary>
+        [TestMethod]
+        public void TestToTextFileForTask()
+        {
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(new string[] { typeof(HelloTask).Assembly.GetName().Name });
+            _serializer.ToTextFile(ns, "avroTask.bin");
+            Assert.IsTrue(File.Exists("avroTask.bin"));
+        }
+
+        /// <summary>
+        /// Test serialize a class hierarchy into a JSon string
+        /// </summary>
+        [TestMethod]
+        public void TestToString()
+        {
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(new string[] { typeof(HelloTask).Assembly.GetName().Name });
+            string s = _serializer.ToString(ns);
+            Assert.IsNotNull(s);
+        }
+
+        /// <summary>
+        /// Test serialize a class hierarchy to a Json string then deserialize it
+        /// In desrialization, in ParseSubHierarchy(), exception is thrown: 
+        /// Unable to cast object of type 'Newtonsoft.Json.Linq.JObject' to type 'Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroClassNode'.
+        /// This is because auto generated code use object as return type instead of AvroClassNode
+        /// </summary>
+        [Ignore]  //TODO: after Avro fix the issue. Enable the test
+        [TestMethod]
+        public void TestToFromJsonString()
+        {
+            Type timerType = typeof(Timer);
+            Type secondType = typeof(Timer.Seconds);
+            Type simpleCOnstuctorType = typeof(SimpleConstructors);
+
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(
+                new string[] { typeof(Timer).Assembly.GetName().Name, typeof(SimpleConstructors).Assembly.GetName().Name });
+            IClassNode timerClassNode = (IClassNode)ns.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode = ns.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode = (IClassNode)ns.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            string s = _serializer.ToString(ns);
+            IClassHierarchy ns2 = _serializer.FromString(s);
+
+            IClassNode timerClassNode2 = (IClassNode)ns2.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode2 = ns2.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode2 = (IClassNode)ns2.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            Assert.AreEqual(timerClassNode, timerClassNode2);
+            Assert.AreEqual(secondNode, secondNode2);
+            Assert.AreEqual(simpleConstructorsClassNode, simpleConstructorsClassNode2);
+        }
+
+        /// <summary>
+        /// Test serialize a class hierarchy to a text file then deserialize it
+        /// In desrialization, in ParseSubHierarchy(), exception is thrown: 
+        /// Unable to cast object of type 'Newtonsoft.Json.Linq.JObject' to type 'Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroClassNode'.
+        /// This is because auto generated code use object as return type instead of AvroClassNode
+        /// </summary>
+        [Ignore] //TODO: after Avro fix the issue. Enable the test
+        [TestMethod]
+        public void TestToFromTextFile()
+        {
+            Type timerType = typeof(Timer);
+            Type secondType = typeof(Timer.Seconds);
+            Type simpleCOnstuctorType = typeof(SimpleConstructors);
+
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(
+                new string[] { typeof(Timer).Assembly.GetName().Name, typeof(SimpleConstructors).Assembly.GetName().Name });
+            IClassNode timerClassNode = (IClassNode)ns.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode = ns.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode = (IClassNode)ns.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            _serializer.ToTextFile(ns, "avroEven.txt");
+            IClassHierarchy ns2 = _serializer.FromTextFile("avroEven.txt");
+
+            IClassNode timerClassNode2 = (IClassNode)ns2.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode2 = ns2.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode2 = (IClassNode)ns2.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            Assert.AreEqual(timerClassNode, timerClassNode2);
+            Assert.AreEqual(secondNode, secondNode2);
+            Assert.AreEqual(simpleConstructorsClassNode, simpleConstructorsClassNode2);
+        }
+
+        /// <summary>
+        /// Test serialize a class hierarchy to a fiel and deserialize from the  file
+        /// Currently, in ToFile() method, writer.Write(avronNodeData) throw exception "Value cannot be null.\r\nParameter name: value". 
+        /// </summary>
+        [Ignore]  //TODO: after Avro fix the issue. Enable the test
+        [TestMethod]
+        public void TestToFromFile()
+        {
+            Type timerType = typeof(Timer);
+            Type secondType = typeof(Timer.Seconds);
+            Type simpleCOnstuctorType = typeof(SimpleConstructors);
+
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(
+                new string[] { typeof(Timer).Assembly.GetName().Name, typeof(SimpleConstructors).Assembly.GetName().Name });
+            IClassNode timerClassNode = (IClassNode)ns.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode = ns.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode = (IClassNode)ns.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            _serializer.ToFile(ns, "avroEven.bin");
+            IClassHierarchy ns2 = _serializer.FromFile("avroEven.bin");
+
+            IClassNode timerClassNode2 = (IClassNode)ns2.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode2 = ns2.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode2 = (IClassNode)ns2.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            Assert.AreEqual(timerClassNode, timerClassNode2);
+            Assert.AreEqual(secondNode, secondNode2);
+            Assert.AreEqual(simpleConstructorsClassNode, simpleConstructorsClassNode2);
+        }
+
+        /// <summary>
+        /// Test serialize class hierarchy to byte array and deserializa back to class hierarchy
+        /// AvroSerializer.Serialize(stream, obj) doesn't allow any null values in the obj to be serialized even if it is nullable
+        /// </summary>
+        [Ignore]  //TODO: after Avro fix the issue. Enable the test
+        [TestMethod]
+        public void TestToFromByteArray()
+        {
+            Type timerType = typeof(Timer);
+            Type secondType = typeof(Timer.Seconds);
+            Type simpleCOnstuctorType = typeof(SimpleConstructors);
+
+            IClassHierarchy ns = TangFactory.GetTang().GetClassHierarchy(
+                new string[] { typeof(Timer).Assembly.GetName().Name, typeof(SimpleConstructors).Assembly.GetName().Name });
+            IClassNode timerClassNode = (IClassNode)ns.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode = ns.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode = (IClassNode)ns.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            byte[] b = _serializer.ToByteArray(ns);
+            IClassHierarchy ns2 = _serializer.FromByteArray(b);
+
+            IClassNode timerClassNode2 = (IClassNode)ns2.GetNode(timerType.AssemblyQualifiedName);
+            INode secondNode2 = ns2.GetNode(secondType.AssemblyQualifiedName);
+            IClassNode simpleConstructorsClassNode2 = (IClassNode)ns2.GetNode(simpleCOnstuctorType.AssemblyQualifiedName);
+
+            Assert.AreEqual(timerClassNode, timerClassNode2);
+            Assert.AreEqual(secondNode, secondNode2);
+            Assert.AreEqual(simpleConstructorsClassNode, simpleConstructorsClassNode2);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj b/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
index 04bf80d..6c11856 100644
--- a/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
@@ -36,6 +36,9 @@ under the License.
     <Reference Include="Microsoft.Hadoop.Avro">
       <HintPath>$(PackagesDir)\Microsoft.Hadoop.Avro.$(AvroVersion)\lib\net40\Microsoft.Hadoop.Avro.dll</HintPath>
     </Reference>
+        <Reference Include="Newtonsoft.Json">
+      <HintPath>$(PackagesDir)\Newtonsoft.Json.$(NewtonsoftJsonVersion)\lib\net45\Newtonsoft.Json.dll</HintPath>
+    </Reference>
     <Reference Include="Microsoft.VisualStudio.QualityTools.UnitTestFramework, Version=10.1.0.0, Culture=neutral, PublicKeyToken=b03f5f7f11d50a3a, processorArchitecture=MSIL" />
     <Reference Include="protobuf-net">
       <HintPath>$(PackagesDir)\protobuf-net.$(ProtobufVersion)\lib\net40\protobuf-net.dll</HintPath>
@@ -45,6 +48,7 @@ under the License.
   </ItemGroup>
   <ItemGroup>
     <Compile Include="ClassHierarchy\TestAnonymousType.cs" />
+    <Compile Include="ClassHierarchy\TestAvroSerialization.cs" />
     <Compile Include="ClassHierarchy\TestClassHierarchy.cs" />
     <Compile Include="ClassHierarchy\TestClassHierarchyRoundTrip.cs" />
     <Compile Include="ClassHierarchy\TestGeneric.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchy.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchy.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchy.cs
new file mode 100644
index 0000000..f245c13
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchy.cs
@@ -0,0 +1,291 @@
+/**
+ * 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.Exceptions;
+using Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Types;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy
+{
+    /// <summary>
+    /// Deserialized Class Hierarchy 
+    /// </summary>
+    internal class AvroClassHierarchy : IClassHierarchy
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(AvroClassHierarchy));
+
+        private readonly IPackageNode _rootNode;
+        private readonly IDictionary<string, INode> _lookupTable = new Dictionary<string, INode>();
+
+        /// <summary>
+        /// create a AvroClassHierarchy with empty nodes and lookup table. It can be used to merge other class hierarchy to it
+        /// </summary>
+        internal AvroClassHierarchy()
+        {
+            _rootNode = new PackageNodeImpl();
+        }
+
+        /// <summary>
+        /// Convert AvroNode into AvroClassHierarchy object
+        /// </summary>
+        /// <param name="root"></param>
+        internal AvroClassHierarchy(AvroNode root)
+        {
+            _rootNode = new PackageNodeImpl();
+            if (root.packageNode == null)
+            {
+                Utilities.Diagnostics.Exceptions.Throw(new ArgumentException("Expected a package node.  Got: " + root), LOGGER); 
+            }
+
+            foreach (AvroNode child in root.children)
+            {
+                ParseSubHierarchy(_rootNode, child);
+            }
+            
+            BuildHashTable(_rootNode);
+
+            foreach (AvroNode child in root.children)
+            {
+                WireUpInheritanceRelationships(child);
+            }
+        }
+
+        private void ParseSubHierarchy(INode parent, AvroNode n)
+        {
+            INode parsed = null;
+            if (n.packageNode != null && !n.packageNode.Equals(""))
+            {
+                parsed = new PackageNodeImpl(parent, n.name, n.fullName);
+            }
+            else if (n.namedParameterNode != null && !n.namedParameterNode.Equals(""))
+            {
+                AvroNamedParameterNode np = (AvroNamedParameterNode)n.namedParameterNode;
+                parsed = new NamedParameterNodeImpl(parent, n.name,
+                    n.fullName, np.fullArgClassName, np.simpleArgClassName,
+                    np.isSet, np.isList, np.documentation, np.shortName,
+                    np.instanceDefault.ToArray());
+            }
+            else if (n.classNode != null && !n.classNode.Equals(""))
+            {
+                AvroClassNode cn = (AvroClassNode)n.classNode;
+                IList<IConstructorDef> injectableConstructors = new List<IConstructorDef>();
+                IList<IConstructorDef> allConstructors = new List<IConstructorDef>();
+
+                foreach (AvroConstructorDef injectable in cn.injectableConstructors)
+                {
+                    IConstructorDef def = ParseConstructorDef(injectable, true);
+                    injectableConstructors.Add(def);
+                    allConstructors.Add(def);
+                }
+                foreach (AvroConstructorDef other in cn.otherConstructors)
+                {
+                    IConstructorDef def = ParseConstructorDef(other, false);
+                    allConstructors.Add(def);
+                }
+
+                parsed = new ClassNodeImpl(parent, n.name, n.fullName,
+                cn.isUnit, cn.isInjectionCandidate,
+                cn.isExternalConstructor, injectableConstructors,
+                allConstructors, cn.defaultImplementation);
+            }
+            else
+            {
+                Utilities.Diagnostics.Exceptions.Throw(new IllegalStateException("Bad protocol buffer: got abstract node" + n), LOGGER);
+            }
+
+            foreach (AvroNode child in n.children)
+            {
+                ParseSubHierarchy(parsed, child);
+            }
+        }
+
+        private void WireUpInheritanceRelationships(AvroNode n)
+        {
+            if (n.classNode != null && !n.classNode.Equals(""))
+            {
+                AvroClassNode cn = (AvroClassNode)n.classNode;
+                IClassNode iface = null;
+
+                try
+                {
+                    iface = (IClassNode)GetNode(n.fullName);
+                }
+                catch (NameResolutionException e)
+                {
+                    Utilities.Diagnostics.Exceptions.Caught(e, Level.Error, LOGGER);
+                    var ex = new IllegalStateException("When reading protocol buffer node "
+                        + n.fullName + " does not exist.  Full record is " + n, e);
+                    Utilities.Diagnostics.Exceptions.Throw(ex, LOGGER);
+                }
+
+                foreach (string impl in cn.implFullNames)
+                {
+                    try
+                    {
+                        iface.PutImpl((IClassNode)GetNode(impl));
+                    }
+                    catch (NameResolutionException e)
+                    {
+                        Utilities.Diagnostics.Exceptions.Caught(e, Level.Error, LOGGER);
+                        var ex = new IllegalStateException("When reading protocol buffer node "
+                            + n + " refers to non-existent implementation:" + impl);
+                        Utilities.Diagnostics.Exceptions.Throw(ex, LOGGER);
+
+                    }
+                    catch (InvalidCastException e)
+                    {
+                        Utilities.Diagnostics.Exceptions.Caught(e, Level.Error, LOGGER);
+                        try
+                        {
+                            var ex = new IllegalStateException(
+                                "When reading protocol buffer node " + n
+                                + " found implementation" + GetNode(impl)
+                                + " which is not a ClassNode!");
+                            Utilities.Diagnostics.Exceptions.Throw(ex, LOGGER);
+                        }
+                        catch (NameResolutionException ne)
+                        {
+                            Utilities.Diagnostics.Exceptions.Caught(ne, Level.Error, LOGGER);
+                            var ex = new IllegalStateException(
+                                "Got 'cant happen' exception when producing error message for " + e);
+                            Utilities.Diagnostics.Exceptions.Throw(ex, LOGGER);
+                        }
+                    }
+                }
+            }
+        }
+
+        /// <summary>
+        /// Build hashtable to index the node
+        /// </summary>
+        /// <param name="n"></param>
+        public void BuildHashTable(INode n)
+        {
+            foreach (INode child in n.GetChildren())
+            {
+                _lookupTable.Add(child.GetFullName(), child);
+                BuildHashTable(child);
+            }
+        }
+
+        /// <summary>
+        /// Get a Node from the class hierarchy 
+        /// </summary>
+        /// <param name="fullName"></param>
+        /// <returns></returns>
+        public INode GetNode(string fullName)
+        {
+            INode ret;
+            _lookupTable.TryGetValue(fullName, out ret);
+            if (ret == null)
+            {
+                var ex = new NameResolutionException(fullName, "Cannot resolve the name from the class hierarchy during deserialization: " + fullName);
+                Utilities.Diagnostics.Exceptions.Throw(ex, LOGGER);
+            }
+            return ret;
+        }
+
+        /// <summary>
+        /// return root node
+        /// </summary>
+        /// <returns></returns>
+        public INode GetNamespace()
+        {
+            return _rootNode;
+        }
+
+        /// <summary>
+        /// Check if impl is an implementation of inter
+        /// </summary>
+        /// <param name="inter"></param>
+        /// <param name="impl"></param>
+        /// <returns></returns>
+        public bool IsImplementation(IClassNode inter, IClassNode impl)
+        {
+            return impl.IsImplementationOf(inter);
+        }
+
+        /// <summary>
+        /// Merge two Class Hierarchy 
+        /// </summary>
+        /// <param name="ch"></param>
+        /// <returns></returns>
+        public IClassHierarchy Merge(IClassHierarchy ch)
+        {
+            if (this == ch)
+            {
+                return this;
+            }
+
+            if (!(ch is AvroClassHierarchy))
+            {
+                Utilities.Diagnostics.Exceptions.Throw(new NotSupportedException(
+                                                            "Cannot merge ExternalClassHierarchies yet!"), LOGGER);
+            }
+
+            AvroClassHierarchy ach = (AvroClassHierarchy)ch;
+            foreach (var pair in ach._lookupTable)
+            {
+                if (!this._lookupTable.ContainsKey(pair.Key))
+                {
+                    this._lookupTable.Add(pair);
+                }
+            }
+
+            foreach (INode n in ch.GetNamespace().GetChildren())
+            {
+                if (!_rootNode.Contains(n.GetFullName()))
+                {
+                    if (n is INamedParameterNode)
+                    {
+                        INamedParameterNode np = (INamedParameterNode)n;
+                        new NamedParameterNodeImpl(this._rootNode, np.GetName(),
+                                                   np.GetFullName(), np.GetFullArgName(), np.GetSimpleArgName(),
+                                                   np.IsSet(), np.IsList(), np.GetDocumentation(), np.GetShortName(),
+                                                   np.GetDefaultInstanceAsStrings().ToArray());
+                    }
+                    else if (n is IClassNode)
+                    {
+                        IClassNode cn = (IClassNode)n;
+                        new ClassNodeImpl(_rootNode, cn.GetName(), cn.GetFullName(),
+                                          cn.IsUnit(), cn.IsInjectionCandidate(),
+                                          cn.IsExternalConstructor(), cn.GetInjectableConstructors(),
+                                          cn.GetAllConstructors(), cn.GetDefaultImplementation());
+                    }
+                }
+            }
+            return this;
+        }
+
+        private IConstructorDef ParseConstructorDef(AvroConstructorDef def, bool isInjectable)
+        {
+            IList<IConstructorArg> args = new List<IConstructorArg>();
+            foreach (AvroConstructorArg arg in def.constructorArgs)
+            {
+                args.Add(new ConstructorArgImpl(arg.fullArgClassName, arg.namedParameterName, arg.isInjectionFuture));
+            }
+            return new ConstructorDefImpl(def.fullClassName, args.ToArray(), isInjectable);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchySerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchySerializer.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchySerializer.cs
new file mode 100644
index 0000000..f1f8000
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroClassHierarchySerializer.cs
@@ -0,0 +1,483 @@
+/**
+ * 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 System.Linq;
+using System.Runtime.Serialization;
+using System.Text;
+using Microsoft.Hadoop.Avro;
+using Microsoft.Hadoop.Avro.Container;
+using Newtonsoft.Json;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Exceptions;
+using Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Types;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy
+{
+    /// <summary>
+    /// AvroClassHierarchySerializer is to serialize and deserialize ClassHierarchy
+    /// </summary>
+    public class AvroClassHierarchySerializer : IClassHierarchySerializer
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(AvroClassHierarchySerializer));
+
+        /// <summary>
+        /// Default constructor for the interface IClassHierarchySerializer
+        /// </summary>
+        [Inject]
+        private AvroClassHierarchySerializer()
+        {
+        }
+
+        /// <summary>
+        /// Avro schema
+        /// </summary>
+        /// <returns></returns>
+        private string GetSchema()
+        {
+            var serializer = AvroSerializer.Create<AvroNode>();
+            return serializer.WriterSchema.ToString();
+        }
+
+        /// <summary>
+        /// Serialize a ClassHierarchy into a file 
+        /// </summary>
+        /// <param name="c"></param>
+        /// <param name="fileName"></param>
+        public void ToFile(IClassHierarchy c, string fileName)
+        {
+            var avronNodeData = ToAvroNode(c);
+            using (var buffer = new MemoryStream())
+            {
+                using (var w = AvroContainer.CreateWriter<AvroNode>(buffer, Codec.Null))
+                {
+                    using (var writer = new SequentialWriter<AvroNode>(w, 24))
+                    {
+                        writer.Write(avronNodeData);
+                    }
+                }
+
+                if (!WriteFile(buffer, fileName))
+                {
+                    var e = new ApplicationException("Error during file operation. Quitting method: " + fileName);
+                    Utilities.Diagnostics.Exceptions.Throw(e, LOGGER);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Serialize a ClassHierarchy into a text file as Json string
+        /// </summary>
+        /// <param name="c"></param>
+        /// <param name="fileName"></param>
+        public void ToTextFile(IClassHierarchy c, string fileName)
+        {
+            var fp = new StreamWriter(fileName);
+            fp.WriteLine(ToString(c));
+            fp.Close();
+        }
+
+        /// <summary>
+        /// erialize a ClassHierarchy into a byte array
+        /// </summary>
+        /// <param name="c"></param>
+        /// <returns></returns>
+        public byte[] ToByteArray(IClassHierarchy c)
+        {
+            AvroNode obj = ToAvroNode(c);
+            return AvroSerialize(obj);
+        }
+
+        /// <summary>
+        /// Serialize a ClassHierarchy into a Json string
+        /// </summary>
+        /// <param name="c"></param>
+        /// <returns></returns>
+        public string ToString(IClassHierarchy c)
+        {
+            AvroNode obj = ToAvroNode(c);
+            string s = JsonConvert.SerializeObject(obj, Formatting.Indented);
+            return s;
+        }
+
+        /// <summary>
+        /// Deserailize a ClassHierarchy from a file
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public IClassHierarchy FromFile(string fileName)
+        {
+            AvroNode avroNode = AvroDeserializeFromFile(fileName);
+            return FromAvroNode(avroNode);
+        }
+
+        /// <summary>
+        /// Get Json string from the text file, the deserailize it into ClassHierarchy
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public IClassHierarchy FromTextFile(string fileName)
+        {
+            string line;
+            StringBuilder b = new StringBuilder();
+
+            StreamReader file = new StreamReader(fileName);
+            while ((line = file.ReadLine()) != null)
+            {
+                b.Append(line);
+            }
+            file.Close();
+
+            return FromString(b.ToString());
+        }
+
+        /// <summary>
+        /// Deserailize a ClassHierarchy from a byte array
+        /// </summary>
+        /// <param name="bytes"></param>
+        /// <returns></returns>
+        public IClassHierarchy FromByteArray(byte[] bytes)
+        {
+            AvroNode avroNode = AvroDeserialize(bytes);
+            return FromAvroNode(avroNode);
+        }
+
+        /// <summary>
+        /// Deserailize a ClassHierarchy from a Json string
+        /// </summary>
+        /// <param name="jsonString"></param>
+        /// <returns></returns>
+        public IClassHierarchy FromString(string jsonString)
+        {
+            AvroNode avroConf = JsonConvert.DeserializeObject<AvroNode>(jsonString);
+            return FromAvroNode(avroConf);
+        }
+
+        /// <summary>
+        /// Serialize a ClassHierarchy into AvroNode object
+        /// </summary>
+        /// <param name="ch"></param>
+        /// <returns></returns>
+        public AvroNode ToAvroNode(IClassHierarchy ch)
+        {
+            return NewAvroNode(ch.GetNamespace());
+        }
+
+        /// <summary>
+        /// Deserailize ClassHierarchy from an AvroNode into AvroClassHierarchy object
+        /// </summary>
+        /// <param name="n"></param>
+        /// <returns></returns>
+        public IClassHierarchy FromAvroNode(AvroNode n)
+        {
+            return new AvroClassHierarchy(n);
+        }
+
+        private AvroNode NewAvroNode(INode n)
+        {
+            IList<AvroNode> children = new List<AvroNode>();
+
+            foreach (INode child in n.GetChildren())
+            {
+                children.Add(NewAvroNode(child));
+            }
+
+            if (n is IClassNode)
+            {
+                IClassNode cn = (IClassNode) n;
+                IList<IConstructorDef> injectable = cn.GetInjectableConstructors();
+                IList<IConstructorDef> all = cn.GetAllConstructors();
+                IList<IConstructorDef> others = new List<IConstructorDef>(all);
+
+                foreach (var c in injectable)
+                {
+                    others.Remove(c);
+                }
+
+                IList<AvroConstructorDef> injectableConstructors = new List<AvroConstructorDef>();
+                foreach (IConstructorDef inj in injectable)
+                {
+                    injectableConstructors.Add(NewConstructorDef(inj));
+                }
+
+                IList<AvroConstructorDef> otherConstructors = new List<AvroConstructorDef>();
+                foreach (IConstructorDef other in others)
+                {
+                    otherConstructors.Add(NewConstructorDef(other));
+                }
+
+                List<string> implFullNames = new List<string>();
+                foreach (IClassNode impl in cn.GetKnownImplementations())
+                {
+                    implFullNames.Add(impl.GetFullName()); //we use class fully qualifed name 
+                }
+
+                return NewClassNode(cn.GetName(), cn.GetFullName(),
+                    cn.IsInjectionCandidate(), cn.IsExternalConstructor(), cn.IsUnit(),
+                    injectableConstructors, otherConstructors, implFullNames, children);
+            }
+
+            if (n is INamedParameterNode)
+            {
+                INamedParameterNode np = (INamedParameterNode) n;
+                return NewNamedParameterNode(np.GetName(), np.GetFullName(),
+                    np.GetSimpleArgName(), np.GetFullArgName(), np.IsSet(), np.IsList(), np.GetDocumentation(),
+                    np.GetShortName(), np.GetDefaultInstanceAsStrings(), children);
+            }
+
+            if (n is IPackageNode)
+            {
+                return NewPackageNode(n.GetName(), n.GetFullName(), children);
+            }
+
+            Utilities.Diagnostics.Exceptions.Throw(
+                new IllegalStateException("Encountered unknown type of Node: " + n), LOGGER);
+            return null;
+        }
+
+        private AvroNode NewClassNode(String name,
+            String fullName,
+            bool isInjectionCandidate,
+            bool isExternalConstructor, bool isUnit,
+            IList<AvroConstructorDef> injectableConstructors,
+            IList<AvroConstructorDef> otherConstructors,
+            IList<String> implFullNames, IList<AvroNode> children)
+        {
+            AvroClassNode classNode = new AvroClassNode();
+
+            classNode.isInjectionCandidate = isInjectionCandidate;
+            classNode.injectableConstructors = new List<AvroConstructorDef>();
+            foreach (var ic in injectableConstructors)
+            {
+                classNode.injectableConstructors.Add(ic);
+            }
+
+            classNode.otherConstructors = new List<AvroConstructorDef>();
+            foreach (var oc in otherConstructors)
+            {
+                classNode.otherConstructors.Add(oc);
+            }
+
+            classNode.implFullNames = new List<string>();
+            foreach (var implFullName in implFullNames)
+            {
+                classNode.implFullNames.Add(implFullName);
+            }
+
+            AvroNode n = new AvroNode();
+            n.name = name;
+            n.fullName = fullName;
+            n.classNode = classNode;
+
+            n.children = new List<AvroNode>();
+            foreach (var c in children)
+            {
+                n.children.Add(c);
+            }
+
+            return n;
+        }
+
+        private AvroNode NewNamedParameterNode(string name,
+            string fullName, string simpleArgClassName, string fullArgClassName,
+            bool isSet, bool isList, string documentation, // can be null
+            string shortName, // can be null
+            string[] instanceDefault, // can be null
+            IList<AvroNode> children)
+        {
+            AvroNamedParameterNode namedParameterNode = new AvroNamedParameterNode();
+            namedParameterNode.simpleArgClassName = simpleArgClassName;
+            namedParameterNode.fullArgClassName = fullArgClassName;
+            namedParameterNode.isSet = isSet;
+            namedParameterNode.isList = isList;
+
+            if (documentation != null)
+            {
+                namedParameterNode.documentation = documentation;
+            }
+
+            if (shortName != null)
+            {
+                namedParameterNode.shortName = shortName;
+            }
+
+            namedParameterNode.instanceDefault = new List<string>();
+            foreach (var id in instanceDefault)
+            {
+                namedParameterNode.instanceDefault.Add(id);
+            }
+
+            AvroNode n = new AvroNode();
+            n.name = name;
+            n.fullName = fullName;
+            n.namedParameterNode = namedParameterNode;
+
+            n.children = new List<AvroNode>();
+            foreach (var c in children)
+            {
+                n.children.Add(c);
+            }
+
+            return n;
+        }
+
+        private AvroNode NewPackageNode(string name,
+            string fullName, IList<AvroNode> children)
+        {
+            AvroPackageNode packageNode = new AvroPackageNode();
+            AvroNode n = new AvroNode();
+            n.name = name;
+            n.fullName = fullName;
+            n.packageNode = packageNode;
+
+            n.children = new List<AvroNode>();
+            foreach (var c in children)
+            {
+                n.children.Add(c);
+            }
+
+            return n;
+        }
+
+        private AvroConstructorDef NewConstructorDef(IConstructorDef def)
+        {
+            IList<AvroConstructorArg> args = new List<AvroConstructorArg>();
+            foreach (IConstructorArg arg in def.GetArgs())
+            {
+                args.Add(NewConstructorArg(arg.Gettype(), arg.GetNamedParameterName(), arg.IsInjectionFuture()));
+            }
+
+            AvroConstructorDef constDef = new AvroConstructorDef();
+            constDef.fullClassName = def.GetClassName();
+
+            constDef.constructorArgs = new List<AvroConstructorArg>();
+            foreach (AvroConstructorArg arg in args)
+            {
+                constDef.constructorArgs.Add(arg);
+            }
+
+            return constDef;
+        }
+
+        private AvroConstructorArg NewConstructorArg(string fullArgClassName, string namedParameterName,
+            bool isFuture)
+        {
+            AvroConstructorArg constArg = new AvroConstructorArg();
+            constArg.fullArgClassName = fullArgClassName;
+            constArg.namedParameterName = namedParameterName;
+            constArg.isInjectionFuture = isFuture;
+            return constArg;
+        }
+
+        private AvroNode AvroDeserializeFromFile(string fileName)
+        {
+            AvroNode avroNode = null;
+            try
+            {
+                using (var buffer = new MemoryStream())
+                {
+                    if (!ReadFile(buffer, fileName))
+                    {
+                        var e =
+                            new ApplicationException("Error during file operation. Quitting method : " + fileName);
+                        Utilities.Diagnostics.Exceptions.Throw(e, LOGGER);
+                    }
+
+                    buffer.Seek(0, SeekOrigin.Begin);
+                    using (
+                        var reader =
+                            new SequentialReader<AvroNode>(AvroContainer.CreateReader<AvroNode>(buffer, true)))
+                    {
+                        var results = reader.Objects;
+
+                        if (results != null)
+                        {
+                            avroNode = (AvroNode)results.First();
+                        }
+                    }
+                }
+            }
+            catch (SerializationException ex)
+            {
+                Utilities.Diagnostics.Exceptions.Caught(ex, Level.Error, LOGGER);
+                var e = new ApplicationException("Cannot deserialize the file: " + fileName, ex);
+                Utilities.Diagnostics.Exceptions.Throw(e, LOGGER);
+            }
+
+            return avroNode;
+        }
+
+        private bool WriteFile(MemoryStream inputStream, string path)
+        {
+            try
+            {
+                using (FileStream fs = File.Create(path))
+                {
+                    inputStream.WriteTo(fs);
+                }
+                return true;
+            }
+            catch (Exception e)
+            {
+                Utilities.Diagnostics.Exceptions.Caught(e, Level.Error, LOGGER);
+                return false;
+            }
+        }
+
+        private bool ReadFile(MemoryStream outputStream, string path)
+        {
+            try
+            {
+                byte[] data = File.ReadAllBytes(path);
+                outputStream.Write(data, 0, data.Length);
+                return true;
+            }
+            catch (Exception e)
+            {
+                Org.Apache.REEF.Utilities.Diagnostics.Exceptions.Caught(e, Level.Error, LOGGER);
+                return false;
+            }
+        }
+
+        private byte[] AvroSerialize(AvroNode obj)
+        {
+            var serializer = AvroSerializer.Create<AvroNode>();
+            using (MemoryStream stream = new MemoryStream())
+            {
+                serializer.Serialize(stream, obj);
+                return stream.GetBuffer();
+            }
+        }
+
+        private AvroNode AvroDeserialize(byte[] serializedBytes)
+        {
+            var serializer = AvroSerializer.Create<AvroNode>();
+
+            using (var stream = new MemoryStream(serializedBytes))
+            {
+                return serializer.Deserialize(stream);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroClassNode.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroClassNode.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroClassNode.cs
new file mode 100644
index 0000000..2cb40a7
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroClassNode.cs
@@ -0,0 +1,120 @@
+/**
+ * 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.
+ */
+
+//<auto-generated />
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.tang.implementation.avro.AvroClassNode.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.tang.implementation.avro")]
+    [KnownType(typeof(List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorDef>))]
+    [KnownType(typeof(List<string>))]
+    public partial class AvroClassNode
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroClassNode"",""fields"":[{""name"":""isInjectionCandidate"",""type"":""boolean""},{""name"":""isExternalConstructor"",""type"":""boolean""},{""name"":""isUnit"",""type"":""boolean""},{""name"":""injectableConstructors"",""type"":{""type"":""array"",""items"":{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroConstructorDef"",""fields"":[{""name"":""fullClassName"",""type"":""string""},{""name"":""constructorArgs"",""type"":{""type"":""array"",""items"":{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroConstructorArg"",""fields"":[{""name"":""fullArgClassName"",""type"":""string""},{""name"":""namedParameterName"",""type"":[""null"",""string""]},{""name"":""isInjectionFuture"",""type"":""boolean""}]}}}]}}},{""name"":""otherConstructors"",""type"":{""type"":""array"",""items"":""org.apache.reef.tang.implementation.
 avro.AvroConstructorDef""}},{""name"":""implFullNames"",""type"":{""type"":""array"",""items"":""string""}},{""name"":""defaultImplementation"",""type"":[""null"",""string""]}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the isInjectionCandidate field.
+        /// </summary>
+        [DataMember]
+        public bool isInjectionCandidate { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the isExternalConstructor field.
+        /// </summary>
+        [DataMember]
+        public bool isExternalConstructor { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the isUnit field.
+        /// </summary>
+        [DataMember]
+        public bool isUnit { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the injectableConstructors field.
+        /// </summary>
+        [DataMember]
+        public List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorDef> injectableConstructors { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the otherConstructors field.
+        /// </summary>
+        [DataMember]
+        public List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorDef> otherConstructors { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the implFullNames field.
+        /// </summary>
+        [DataMember]
+        public List<string> implFullNames { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the defaultImplementation field.
+        /// </summary>
+        [DataMember]
+        public string defaultImplementation { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroClassNode"/> class.
+        /// </summary>
+        public AvroClassNode()
+        {
+            this.defaultImplementation = null;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroClassNode"/> class.
+        /// </summary>
+        /// <param name="isInjectionCandidate">The isInjectionCandidate.</param>
+        /// <param name="isExternalConstructor">The isExternalConstructor.</param>
+        /// <param name="isUnit">The isUnit.</param>
+        /// <param name="injectableConstructors">The injectableConstructors.</param>
+        /// <param name="otherConstructors">The otherConstructors.</param>
+        /// <param name="implFullNames">The implFullNames.</param>
+        /// <param name="defaultImplementation">The defaultImplementation.</param>
+        public AvroClassNode(bool isInjectionCandidate, bool isExternalConstructor, bool isUnit, List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorDef> injectableConstructors, List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorDef> otherConstructors, List<string> implFullNames, string defaultImplementation)
+        {
+            this.isInjectionCandidate = isInjectionCandidate;
+            this.isExternalConstructor = isExternalConstructor;
+            this.isUnit = isUnit;
+            this.injectableConstructors = injectableConstructors;
+            this.otherConstructors = otherConstructors;
+            this.implFullNames = implFullNames;
+            this.defaultImplementation = defaultImplementation;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorArg.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorArg.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorArg.cs
new file mode 100644
index 0000000..8a0d95b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorArg.cs
@@ -0,0 +1,86 @@
+/**
+ * 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.
+ */
+
+//<auto-generated />
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.tang.implementation.avro.AvroConstructorArg.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.tang.implementation.avro")]
+    public partial class AvroConstructorArg
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroConstructorArg"",""fields"":[{""name"":""fullArgClassName"",""type"":""string""},{""name"":""namedParameterName"",""type"":[""null"",""string""]},{""name"":""isInjectionFuture"",""type"":""boolean""}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the fullArgClassName field.
+        /// </summary>
+        [DataMember]
+        public string fullArgClassName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the namedParameterName field.
+        /// </summary>
+        [DataMember]
+        public string namedParameterName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the isInjectionFuture field.
+        /// </summary>
+        [DataMember]
+        public bool isInjectionFuture { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroConstructorArg"/> class.
+        /// </summary>
+        public AvroConstructorArg()
+        {
+            this.namedParameterName = null;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroConstructorArg"/> class.
+        /// </summary>
+        /// <param name="fullArgClassName">The fullArgClassName.</param>
+        /// <param name="namedParameterName">The namedParameterName.</param>
+        /// <param name="isInjectionFuture">The isInjectionFuture.</param>
+        public AvroConstructorArg(string fullArgClassName, string namedParameterName, bool isInjectionFuture)
+        {
+            this.fullArgClassName = fullArgClassName;
+            this.namedParameterName = namedParameterName;
+            this.isInjectionFuture = isInjectionFuture;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorDef.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorDef.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorDef.cs
new file mode 100644
index 0000000..06f6fda
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroConstructorDef.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.
+ */
+
+//<auto-generated />
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.tang.implementation.avro.AvroConstructorDef.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.tang.implementation.avro")]
+    [KnownType(typeof(List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorArg>))]
+    public partial class AvroConstructorDef
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroConstructorDef"",""fields"":[{""name"":""fullClassName"",""type"":""string""},{""name"":""constructorArgs"",""type"":{""type"":""array"",""items"":{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroConstructorArg"",""fields"":[{""name"":""fullArgClassName"",""type"":""string""},{""name"":""namedParameterName"",""type"":[""null"",""string""]},{""name"":""isInjectionFuture"",""type"":""boolean""}]}}}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the fullClassName field.
+        /// </summary>
+        [DataMember]
+        public string fullClassName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the constructorArgs field.
+        /// </summary>
+        [DataMember]
+        public List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorArg> constructorArgs { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroConstructorDef"/> class.
+        /// </summary>
+        public AvroConstructorDef()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroConstructorDef"/> class.
+        /// </summary>
+        /// <param name="fullClassName">The fullClassName.</param>
+        /// <param name="constructorArgs">The constructorArgs.</param>
+        public AvroConstructorDef(string fullClassName, List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroConstructorArg> constructorArgs)
+        {
+            this.fullClassName = fullClassName;
+            this.constructorArgs = constructorArgs;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNamedParameterNode.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNamedParameterNode.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNamedParameterNode.cs
new file mode 100644
index 0000000..2f3f356
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNamedParameterNode.cs
@@ -0,0 +1,120 @@
+/**
+ * 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.
+ */
+
+//<auto-generated />
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.tang.implementation.avro.AvroNamedParameterNode.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.tang.implementation.avro")]
+    [KnownType(typeof(List<string>))]
+    public partial class AvroNamedParameterNode
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroNamedParameterNode"",""fields"":[{""name"":""simpleArgClassName"",""type"":""string""},{""name"":""fullArgClassName"",""type"":""string""},{""name"":""isSet"",""type"":""boolean""},{""name"":""isList"",""type"":""boolean""},{""name"":""documentation"",""type"":[""null"",""string""]},{""name"":""shortName"",""type"":[""null"",""string""]},{""name"":""instanceDefault"",""type"":{""type"":""array"",""items"":""string""}}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the simpleArgClassName field.
+        /// </summary>
+        [DataMember]
+        public string simpleArgClassName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the fullArgClassName field.
+        /// </summary>
+        [DataMember]
+        public string fullArgClassName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the isSet field.
+        /// </summary>
+        [DataMember]
+        public bool isSet { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the isList field.
+        /// </summary>
+        [DataMember]
+        public bool isList { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the documentation field.
+        /// </summary>
+        [DataMember]
+        public string documentation { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the shortName field.
+        /// </summary>
+        [DataMember]
+        public string shortName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the instanceDefault field.
+        /// </summary>
+        [DataMember]
+        public List<string> instanceDefault { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroNamedParameterNode"/> class.
+        /// </summary>
+        public AvroNamedParameterNode()
+        {
+            this.documentation = null;
+            this.shortName = null;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroNamedParameterNode"/> class.
+        /// </summary>
+        /// <param name="simpleArgClassName">The simpleArgClassName.</param>
+        /// <param name="fullArgClassName">The fullArgClassName.</param>
+        /// <param name="isSet">The isSet.</param>
+        /// <param name="isList">The isList.</param>
+        /// <param name="documentation">The documentation.</param>
+        /// <param name="shortName">The shortName.</param>
+        /// <param name="instanceDefault">The instanceDefault.</param>
+        public AvroNamedParameterNode(string simpleArgClassName, string fullArgClassName, bool isSet, bool isList, string documentation, string shortName, List<string> instanceDefault)
+        {
+            this.simpleArgClassName = simpleArgClassName;
+            this.fullArgClassName = fullArgClassName;
+            this.isSet = isSet;
+            this.isList = isList;
+            this.documentation = documentation;
+            this.shortName = shortName;
+            this.instanceDefault = instanceDefault;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNode.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNode.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNode.cs
new file mode 100644
index 0000000..b38d489
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroNode.cs
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+//<auto-generated />
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.tang.implementation.avro.AvroNode.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.tang.implementation.avro")]
+    [KnownType(typeof(List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroNode>))]
+    public partial class AvroNode
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroNode"",""fields"":[{""name"":""name"",""type"":""string""},{""name"":""fullName"",""type"":""string""},{""name"":""classNode"",""type"":[""null"",{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroClassNode"",""fields"":[{""name"":""isInjectionCandidate"",""type"":""boolean""},{""name"":""isExternalConstructor"",""type"":""boolean""},{""name"":""isUnit"",""type"":""boolean""},{""name"":""injectableConstructors"",""type"":{""type"":""array"",""items"":{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroConstructorDef"",""fields"":[{""name"":""fullClassName"",""type"":""string""},{""name"":""constructorArgs"",""type"":{""type"":""array"",""items"":{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroConstructorArg"",""fields"":[{""name"":""fullArgClassName"",""type"":""string""},{""name"":""n
 amedParameterName"",""type"":[""null"",""string""]},{""name"":""isInjectionFuture"",""type"":""boolean""}]}}}]}}},{""name"":""otherConstructors"",""type"":{""type"":""array"",""items"":""org.apache.reef.tang.implementation.avro.AvroConstructorDef""}},{""name"":""implFullNames"",""type"":{""type"":""array"",""items"":""string""}},{""name"":""defaultImplementation"",""type"":[""null"",""string""]}]}]},{""name"":""namedParameterNode"",""type"":[""null"",{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroNamedParameterNode"",""fields"":[{""name"":""simpleArgClassName"",""type"":""string""},{""name"":""fullArgClassName"",""type"":""string""},{""name"":""isSet"",""type"":""boolean""},{""name"":""isList"",""type"":""boolean""},{""name"":""documentation"",""type"":[""null"",""string""]},{""name"":""shortName"",""type"":[""null"",""string""]},{""name"":""instanceDefault"",""type"":{""type"":""array"",""items"":""string""}}]}]},{""name"":""packageNode"",""type"":[""n
 ull"",{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroPackageNode"",""fields"":[]}]},{""name"":""children"",""type"":{""type"":""array"",""items"":""org.apache.reef.tang.implementation.avro.AvroNode""}}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the name field.
+        /// </summary>
+        [DataMember]
+        public string name { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the fullName field.
+        /// </summary>
+        [DataMember]
+        public string fullName { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the classNode field.
+        /// </summary>
+        [DataMember]
+        [AvroUnion(typeof(AvroNull), typeof(Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroClassNode))]
+        public object classNode { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the namedParameterNode field.
+        /// </summary>
+        [DataMember]
+        [AvroUnion(typeof(AvroNull), typeof(Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroNamedParameterNode))]
+        public object namedParameterNode { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the packageNode field.
+        /// </summary>
+        [DataMember]
+        [AvroUnion(typeof(AvroNull), typeof(Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroPackageNode))]
+        public object packageNode { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the children field.
+        /// </summary>
+        [DataMember]
+        public List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroNode> children { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroNode"/> class.
+        /// </summary>
+        public AvroNode()
+        {
+            this.classNode = null;
+            this.namedParameterNode = null;
+            this.packageNode = null;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroNode"/> class.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        /// <param name="fullName">The fullName.</param>
+        /// <param name="classNode">The classNode.</param>
+        /// <param name="namedParameterNode">The namedParameterNode.</param>
+        /// <param name="packageNode">The packageNode.</param>
+        /// <param name="children">The children.</param>
+        public AvroNode(string name, string fullName, object classNode, object namedParameterNode, object packageNode, List<Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract.AvroNode> children)
+        {
+            this.name = name;
+            this.fullName = fullName;
+            this.classNode = classNode;
+            this.namedParameterNode = namedParameterNode;
+            this.packageNode = packageNode;
+            this.children = children;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroPackageNode.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroPackageNode.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroPackageNode.cs
new file mode 100644
index 0000000..e26a8ca
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/AvroDataContract/AvroPackageNode.cs
@@ -0,0 +1,54 @@
+/**
+ * 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.
+ */
+
+//<auto-generated />
+namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy.AvroDataContract
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.tang.implementation.avro.AvroPackageNode.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.tang.implementation.avro")]
+    public partial class AvroPackageNode
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.tang.implementation.avro.AvroPackageNode"",""fields"":[]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+        
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroPackageNode"/> class.
+        /// </summary>
+        public AvroPackageNode()
+        {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Interface/IClassHierarchySerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Interface/IClassHierarchySerializer.cs b/lang/cs/Org.Apache.REEF.Tang/Interface/IClassHierarchySerializer.cs
new file mode 100644
index 0000000..da554e4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Interface/IClassHierarchySerializer.cs
@@ -0,0 +1,84 @@
+/**
+ * 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 Org.Apache.REEF.Tang.Implementations.ClassHierarchy;
+
+namespace Org.Apache.REEF.Tang.Interface
+{
+    [DefaultImplementation(typeof(AvroClassHierarchySerializer), "default")]
+    public interface IClassHierarchySerializer
+    {
+        /// <summary>
+        /// Serialize a ClassHierarchy into a file 
+        /// </summary>
+        /// <param name="c"></param>
+        /// <param name="fileName"></param>
+        void ToFile(IClassHierarchy c, string fileName);
+
+        /// <summary>
+        /// Serialize a ClassHierarchy into a text file as Jason string
+        /// </summary>
+        /// <param name="c"></param>
+        /// <param name="fileName"></param>
+        void ToTextFile(IClassHierarchy c, string fileName);
+
+        /// <summary>
+        /// erialize a ClassHierarchy into a Jason string
+        /// </summary>
+        /// <param name="c"></param>
+        /// <returns></returns>
+        string ToString(IClassHierarchy c);
+
+        /// <summary>
+        /// erialize a ClassHierarchy into a byte array
+        /// </summary>
+        /// <param name="c"></param>
+        /// <returns></returns>
+        byte[] ToByteArray(IClassHierarchy c);
+
+        /// <summary>
+        /// Deserailize a ClassHierarchy from a file
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        IClassHierarchy FromFile(string fileName);
+
+        /// <summary>
+        /// Deserailize a ClassHierarchy from a text file
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        IClassHierarchy FromTextFile(string fileName);
+
+        /// <summary>
+        /// Deserailize a ClassHierarchy from a byte array
+        /// </summary>
+        /// <param name="bytes"></param>
+        /// <returns></returns>
+        IClassHierarchy FromByteArray(byte[] bytes);
+
+        /// <summary>
+        /// Deserailize a ClassHierarchy from a Jason string
+        /// </summary>
+        /// <param name="jsonString"></param>
+        /// <returns></returns>
+        IClassHierarchy FromString(string jsonString);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/e7550466/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj b/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
index 8f84149..487e7f7 100644
--- a/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
+++ b/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
@@ -72,6 +72,14 @@ under the License.
     <Compile Include="Formats\RequiredImpl.cs" />
     <Compile Include="Formats\RequiredParameter.cs" />
     <Compile Include="Implementations\ClassHierarchy\AbstractNode.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroClassHierarchy.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroClassHierarchySerializer.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroDataContract\AvroClassNode.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroDataContract\AvroConstructorArg.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroDataContract\AvroConstructorDef.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroDataContract\AvroNamedParameterNode.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroDataContract\AvroNode.cs" />
+    <Compile Include="Implementations\ClassHierarchy\AvroDataContract\AvroPackageNode.cs" />
     <Compile Include="Implementations\ClassHierarchy\ClassHierarchyImpl.cs" />
     <Compile Include="Implementations\ClassHierarchy\ClassNodeImpl.cs" />
     <Compile Include="Implementations\ClassHierarchy\ConstructorArgImpl.cs" />
@@ -98,6 +106,7 @@ under the License.
     <Compile Include="Implementations\Tang\TangImpl.cs" />
     <Compile Include="Interface\IAspect.cs" />
     <Compile Include="Interface\IClassHierarchy.cs" />
+    <Compile Include="Interface\IClassHierarchySerializer.cs" />
     <Compile Include="Interface\IConfiguration.cs" />
     <Compile Include="Interface\IConfigurationBuilder.cs" />
     <Compile Include="Interface\ICsClassHierarchy.cs" />