You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ju...@apache.org on 2015/06/30 21:27:12 UTC

incubator-reef git commit: [REEF-395] Implement ClassHierarchySerializer using Avro on the Java side   This addressed the issue by    * Defining the avro schema for class hierarchy   * Implementing the AvroClassHierarchy * Define ClassHierarchySerializ

Repository: incubator-reef
Updated Branches:
  refs/heads/master cb134fe34 -> 96f9e94fa


[REEF-395] Implement ClassHierarchySerializer using Avro on the Java side
 
This addressed the issue by 
  * Defining the avro schema for class hierarchy
  * Implementing the AvroClassHierarchy
  * Define ClassHierarchySerializer interface and implement an Avro version
  * Rewrite the test code to use serializer
  * updating the class hierarchy serializer for the round trip tests
  * Leave TODO comments for the future issues to resolve when we revisit REEF-148

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

This Closes #237

Author:    Yunseong Lee <yu...@apache.org>


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

Branch: refs/heads/master
Commit: 96f9e94fa4b4017bfc95549dc5f38da0b5cc70d8
Parents: cb134fe
Author: Yunseong Lee <yu...@apache.org>
Authored: Fri Jun 19 17:22:11 2015 +0900
Committer: Julia Wang <jw...@yahoo.com>
Committed: Tue Jun 30 12:24:11 2015 -0700

----------------------------------------------------------------------
 .../org/apache/reef/javabridge/Utilities.java   |   2 +
 .../reef/examples/helloCLR/HelloDriver.java     |   2 +
 .../tang/src/main/avro/class_hierarchy.avsc     | 119 ++++
 .../reef/tang/ClassHierarchySerializer.java     |  96 ++++
 .../implementation/avro/AvroClassHierarchy.java | 244 +++++++++
 .../avro/AvroClassHierarchySerializer.java      | 263 +++++++++
 .../tang/implementation/avro/package-info.java  |  23 +
 .../protobuf/ProtocolBufferClassHierarchy.java  |   8 +
 .../reef/tang/util/ValidateConfiguration.java   |   2 +
 .../tang/ClassHierarchyDeserializationTest.java |  23 +-
 .../tang/implementation/TestClassHierarchy.java |   3 +
 .../avro/TestClassHierarchyRoundTrip.java       | 549 +++++++++++++++++++
 .../protobuf/TestClassHierarchyRoundTrip.java   | 401 --------------
 .../apache/reef/tang/test/RoundTripTest.java    |   5 +-
 14 files changed, 1330 insertions(+), 410 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java b/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java
index 82d94a9..74c1549 100644
--- a/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java
+++ b/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java
@@ -33,6 +33,7 @@ import java.nio.file.Paths;
 
 public final class Utilities {
   public static ClassHierarchy loadClassHierarchy(String classHierarchyFile) {
+    // TODO The file should be created via AvroClassHierarchySerializer
     Path p = Paths.get(classHierarchyFile);
     if (!Files.exists(p)) {
       p = Paths.get(System.getProperty("user.dir") + "/reef/global/" + classHierarchyFile);
@@ -40,6 +41,7 @@ public final class Utilities {
     if (!Files.exists(p)) {
       throw new RuntimeException("cannot find file " + p.toAbsolutePath());
     }
+    // TODO Use the AvroClassHierarchy in place of protobuf
     try (final InputStream chin = new FileInputStream(p.toAbsolutePath().toString())) {
       final ClassHierarchyProto.Node root = ClassHierarchyProto.Node.parseFrom(chin);
       final ClassHierarchy ch = new ProtocolBufferClassHierarchy(root);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-examples-clr/src/main/java/org/apache/reef/examples/helloCLR/HelloDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples-clr/src/main/java/org/apache/reef/examples/helloCLR/HelloDriver.java b/lang/java/reef-examples-clr/src/main/java/org/apache/reef/examples/helloCLR/HelloDriver.java
index 18f4623..29b3126 100644
--- a/lang/java/reef-examples-clr/src/main/java/org/apache/reef/examples/helloCLR/HelloDriver.java
+++ b/lang/java/reef-examples-clr/src/main/java/org/apache/reef/examples/helloCLR/HelloDriver.java
@@ -92,7 +92,9 @@ public final class HelloDriver {
    * @return
    */
   private static ClassHierarchy loadClassHierarchy() {
+    // TODO The file should be created by AvroClassHierarchySerializer
     try (final InputStream chin = new FileInputStream(HelloCLR.CLASS_HIERARCHY_FILENAME)) {
+      // TODO Use AvroClassHierarchySerializer instead
       final ClassHierarchyProto.Node root = ClassHierarchyProto.Node.parseFrom(chin); // A
       final ClassHierarchy ch = new ProtocolBufferClassHierarchy(root);
       return ch;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/main/avro/class_hierarchy.avsc
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/avro/class_hierarchy.avsc b/lang/java/reef-tang/tang/src/main/avro/class_hierarchy.avsc
new file mode 100644
index 0000000..3b26c04
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/main/avro/class_hierarchy.avsc
@@ -0,0 +1,119 @@
+/**
+ * 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.implementation.avro",
+    "type":"record",
+    "name":"AvroConstructorArg",
+    "fields":[
+        {"name":"fullArgClassName", "type":"string"},
+        {"name":"namedParameterName", "type":["null", "string"], "default":null},
+        {"name":"isInjectionFuture", "type":"boolean"}
+    ]
+},
+{
+    "namespace":"org.apache.reef.tang.implementation.avro",
+    "type":"record",
+    "name":"AvroConstructorDef",
+    "fields":[
+        {"name":"fullClassName", "type":"string"},
+        {"name":"constructorArgs", "type":{"type":"array", "items":"AvroConstructorArg"}}
+    ]
+},
+{
+    "namespace":"org.apache.reef.tang.implementation.avro",
+    "type":"record",
+    "name":"AvroNamedParameterNode",
+    "fields":[
+        {"name":"simpleArgClassName", "type":"string"},
+        {"name":"fullArgClassName", "type":"string"},
+        {"name":"isSet", "type":"boolean"},
+        {"name":"isList", "type":"boolean"},
+        {"name":"documentation", "type":["null", "string"], "default":null},
+        {"name":"shortName", "type":["null", "string"], "default":null},
+        {"name":"instanceDefault", "type":{"type":"array","items":"string"}}
+    ]
+},
+{
+    "namespace":"org.apache.reef.tang.implementation.avro",
+    "type":"record",
+    "name":"AvroClassNode",
+    "fields":[
+        {"name":"isInjectionCandidate", "type":"boolean"},
+        {"name":"isExternalConstructor", "type":"boolean"},
+        {"name":"isUnit", "type":"boolean"},
+        {"name":"injectableConstructors", "type":{"type":"array", "items":"AvroConstructorDef"}},
+        {"name":"otherConstructors", "type":{"type":"array", "items":"AvroConstructorDef"}},
+        {"name":"implFullNames", "type":{"type":"array", "items":"string"}},
+        {"name":"defaultImplementation", "type":["null", "string"], "default":null}
+    ]
+},
+/*
+ * Intentionally left blank. Packages don't have any interesting
+ * attributes except their names and children.
+ */
+{
+    "namespace":"org.apache.reef.tang.implementation.avro",
+    "type":"record",
+    "name":"AvroPackageNode",
+    "fields":[]
+},
+/*
+ * Node is the abstract base type for all the data encoded in a class hierarchy.
+ * It is represented as an (un)tagged union, so only one of the three optional
+ * fields can be non-null.
+ */
+{
+    "namespace":"org.apache.reef.tang.implementation.avro",
+    "type":"record",
+    "name":"AvroNode",
+    "fields":[
+        /*
+         * The short name of this node (e.g., a class name without the enclosing
+         * package, or a package name without the name of the enclosing package.
+         */
+        {"name":"name", "type":"string"},
+        /*
+         * For some languages it can be difficult to map from a list of nodes to
+         * the correct short name, so for now, the full name is (redundantly)
+         * encoded here.  In Java, the full name is a bunch of short names that
+         * have been concatenated with "." and/or "$".
+         */
+        {"name":"fullName", "type":"string"},
+
+        /*
+         * Exactly one of the next three fields must be defined. See below for
+         * their documentation.
+         */
+        {"name":"classNode", "type":["null", "AvroClassNode"], "default":null},
+        {"name":"namedParameterNode", "type":["null", "AvroNamedParameterNode"], "default":null},
+        {"name":"packageNode", "type":["null", "AvroPackageNode"], "default":null},
+        /*
+         * Class hierarchy nodes are arranged in a tree that mirrors the language
+         * namespaces / packages that contain class names (e.g., Java packages,
+         * C++ namespaces).
+         *
+         * A NamedParameterNode cannot have any children.  A ClassNode can not have
+         * any package nodes as children.  PackageNodes cannot have PackageNodes as
+         * children.
+         */
+        {"name":"children", "type":{"type":"array","items":"AvroNode"}}
+    ]
+}
+]

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/ClassHierarchySerializer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/ClassHierarchySerializer.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/ClassHierarchySerializer.java
new file mode 100644
index 0000000..1ad4706
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/ClassHierarchySerializer.java
@@ -0,0 +1,96 @@
+/*
+ * 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.
+ */
+package org.apache.reef.tang;
+
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.tang.implementation.avro.AvroClassHierarchySerializer;
+
+import java.io.*;
+
+/**
+ * A base interface for ClassHierarchy serializers.
+ */
+@DefaultImplementation(AvroClassHierarchySerializer.class)
+public interface ClassHierarchySerializer {
+  /**
+   * Writes a ClassHierarchy into a file.
+   *
+   * @param classHierarchy the ClassHierarchy to store
+   * @param file the file to store the ClassHierarchy
+   * @throws IOException if there is an error in the process
+   */
+  void toFile(final ClassHierarchy classHierarchy, final File file) throws IOException;
+
+  /**
+   * Writes a ClassHierarchy into a text file.
+   *
+   * @param classHierarchy the ClassHierarchy to store
+   * @param file the text file to store the ClassHierarchy
+   * @throws IOException if there is an error in the process
+   */
+  void toTextFile(final ClassHierarchy classHierarchy, final File file) throws IOException;
+
+   /**
+   * Serializes a ClassHierarchy as a byte[].
+   *
+   * @param classHierarchy the ClassHierarchy to store
+   * @throws IOException if there is an error in the process
+   */
+  byte[] toByteArray(final ClassHierarchy classHierarchy) throws IOException;
+
+  /**
+   * Serializes a ClassHierarchy as a String.
+   *
+   * @param classHierarchy the ClassHierarchy to store
+   * @throws IOException if there is an error in the process
+   */
+  String toString(final ClassHierarchy classHierarchy) throws IOException;
+
+  /**
+   * Loads a ClassHierarchy from a file created with toFile().
+   *
+   * @param file the File to read from
+   * @throws IOException if the File can't be read or parsed
+   */
+  ClassHierarchy fromFile(final File file) throws IOException;
+
+  /**
+   * Loads a ClassHierarchy from a text file created with toTextFile().
+   *
+   * @param file the File to read from
+   * @throws IOException if the File can't be read or parsed
+   */
+  ClassHierarchy fromTextFile(final File file) throws IOException;
+
+  /**
+   * Deserializes a ClassHierarchy from a byte[] created with toByteArray().
+   *
+   * @param theBytes the byte[] to deserialize
+   * @throws IOException if the byte[] can't be read or parsed
+   */
+  ClassHierarchy fromByteArray(final byte[] theBytes) throws IOException;
+
+  /**
+   * Deserializes a ClassHierarchy from a String created with toString().
+   *
+   * @param theString the String to deserialize
+   * @throws IOException if the String can't be read or parsed
+   */
+  ClassHierarchy fromString(final String theString) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchy.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchy.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchy.java
new file mode 100644
index 0000000..a66f8e4
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchy.java
@@ -0,0 +1,244 @@
+/*
+ * 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.
+ */
+package org.apache.reef.tang.implementation.avro;
+
+import org.apache.reef.tang.ClassHierarchy;
+import org.apache.reef.tang.annotations.NamedParameter;
+import org.apache.reef.tang.exceptions.NameResolutionException;
+import org.apache.reef.tang.implementation.types.*;
+import org.apache.reef.tang.types.*;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * Implementation of ClassHierarchy for Avro
+ */
+final class AvroClassHierarchy implements ClassHierarchy {
+  private final PackageNode namespace;
+  private final HashMap<String, Node> lookupTable = new HashMap<>();
+
+  public AvroClassHierarchy(final AvroNode root) {
+    namespace = new PackageNodeImpl();
+    if (root.getPackageNode() == null) {
+      throw new IllegalArgumentException("Expected a package node. Got: " + root);
+    }
+
+    // Register all the classes.
+    for (final AvroNode child : root.getChildren()) {
+      parseSubHierarchy(namespace, child);
+    }
+    buildLookupTable(namespace);
+
+    // Register the implementations
+    for (final AvroNode child : root.getChildren()) {
+      wireUpInheritanceRelationships(child);
+    }
+  }
+
+  /**
+   * Build a table that maps the name to the corresponding Node recursively.
+   */
+  private void buildLookupTable(final Node n) {
+    for (final Node child : n.getChildren()) {
+      lookupTable.put(child.getFullName(), child);
+      buildLookupTable(child);
+    }
+  }
+
+  /**
+   * Parse the constructor definition.
+   */
+  private ConstructorDef<?> parseConstructorDef(final AvroConstructorDef def, final boolean isInjectable) {
+    final List<ConstructorArg> args = new ArrayList<>();
+    for (final AvroConstructorArg arg : def.getConstructorArgs()) {
+      args.add(new ConstructorArgImpl(getString(arg.getFullArgClassName()), getString(arg.getNamedParameterName()),
+              arg.getIsInjectionFuture()));
+    }
+    return new ConstructorDefImpl<>(getString(def.getFullClassName()), args.toArray(new ConstructorArg[0]),
+            isInjectable);
+
+  }
+
+  /**
+   * Register the classes recursively.
+   */
+  private void parseSubHierarchy(final Node parent, final AvroNode n) {
+    final Node parsed;
+    if (n.getPackageNode() != null) {
+      parsed = new PackageNodeImpl(parent, getString(n.getName()), getString(n.getFullName()));
+    } else if (n.getNamedParameterNode() != null) {
+      final AvroNamedParameterNode np = n.getNamedParameterNode();
+
+
+      parsed = new NamedParameterNodeImpl<>(parent, getString(n.getName()), getString(n.getFullName()),
+              getString(np.getFullArgClassName()), getString(np.getSimpleArgClassName()), np.getIsSet(), np.getIsList(),
+              getString(np.getDocumentation()), getString(np.getShortName()), getStringArray(np.getInstanceDefault()));
+    } else if (n.getClassNode() != null) {
+      final AvroClassNode cn = n.getClassNode();
+      final List<ConstructorDef<?>> injectableConstructors = new ArrayList<>();
+      final List<ConstructorDef<?>> allConstructors = new ArrayList<>();
+
+      for (final AvroConstructorDef injectable : cn.getInjectableConstructors()) {
+        final ConstructorDef<?> def = parseConstructorDef(injectable, true);
+        injectableConstructors.add(def);
+        allConstructors.add(def);
+      }
+      for (final AvroConstructorDef other : cn.getOtherConstructors()) {
+        final ConstructorDef<?> def = parseConstructorDef(other, false);
+        allConstructors.add(def);
+      }
+      @SuppressWarnings("unchecked")
+      final ConstructorDef<Object>[] dummy = new ConstructorDef[0];
+      parsed = new ClassNodeImpl<>(parent, getString(n.getName()), getString(n.getFullName()), cn.getIsUnit(),
+              cn.getIsInjectionCandidate(), cn.getIsExternalConstructor(), injectableConstructors.toArray(dummy),
+              allConstructors.toArray(dummy), getString(cn.getDefaultImplementation()));
+    } else {
+      throw new IllegalStateException("Bad avro node: got abstract node" + n);
+    }
+
+    for (final AvroNode child : n.getChildren()) {
+      parseSubHierarchy(parsed, child);
+    }
+  }
+
+  /**
+   * Register the implementation for the ClassNode recursively.
+   */
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private void wireUpInheritanceRelationships(final AvroNode n) {
+    if (n.getClassNode() != null) {
+      final AvroClassNode cn = n.getClassNode();
+      final ClassNode iface;
+      try {
+        iface = (ClassNode) getNode(getString(n.getFullName()));
+      } catch (NameResolutionException e) {
+        final String errorMessage = new StringBuilder()
+                .append("When reading avro node ").append(n.getFullName())
+                .append(" does not exist.  Full record is ").append(n).toString();
+        throw new IllegalStateException(errorMessage, e);
+      }
+      for (final CharSequence impl : cn.getImplFullNames()) {
+        try {
+          iface.putImpl((ClassNode) getNode(getString(impl)));
+        } catch (NameResolutionException e) {
+          final String errorMessage = new StringBuilder()
+                  .append("When reading avro node ").append(n)
+                  .append(" refers to non-existent implementation:").append(impl).toString();
+          throw new IllegalStateException(errorMessage, e);
+        } catch (ClassCastException e) {
+          try {
+            final String errorMessage = new StringBuilder()
+                    .append("When reading avro node ").append(n).append(" found implementation")
+                    .append(getNode(getString(impl))).append(" which is not a ClassNode!").toString();
+            throw new IllegalStateException(errorMessage, e);
+          } catch (NameResolutionException e2) {
+            final String errorMessage = new StringBuilder()
+                    .append("Got 'cant happen' exception when producing error message for ")
+                    .append(e).toString();
+            throw new IllegalStateException(errorMessage);
+          }
+        }
+      }
+    }
+
+    for (final AvroNode child : n.getChildren()) {
+      wireUpInheritanceRelationships(child);
+    }
+  }
+
+  /**
+   * Get the value of the CharSequence. Return null if the CharSequence is null.
+   */
+  private String getString(final CharSequence charSeq) {
+    if (charSeq == null) {
+      return null;
+    } else {
+      return charSeq.toString();
+    }
+  }
+
+  /**
+   * Convert the CharSequence list into the String array.
+   */
+  private String[] getStringArray(final List<CharSequence> charSeqList) {
+    final int length = charSeqList.size();
+    final String[] stringArray = new String[length];
+    for (int i = 0; i < length; i++) {
+      stringArray[i] = getString(charSeqList.get(i));
+    }
+    return stringArray;
+  }
+
+  @Override
+  public Node getNode(final String fullName) throws NameResolutionException {
+    final Node matchedNode = lookupTable.get(fullName);
+    if (matchedNode != null) {
+      return matchedNode;
+    } else {
+      throw new NameResolutionException(fullName, "");
+    }
+  }
+
+  @Override
+  public boolean isImplementation(final ClassNode<?> inter, final ClassNode<?> impl) {
+    return impl.isImplementationOf(inter);
+  }
+
+  @Override
+  public ClassHierarchy merge(final ClassHierarchy ch) {
+    if (this == ch) {
+      return this;
+    }
+    if (!(ch instanceof AvroClassHierarchy)) {
+      throw new UnsupportedOperationException(
+              "Cannot merge with class hierarchies of type: " + ch.getClass().getName());
+    }
+
+    final AvroClassHierarchy ach = (AvroClassHierarchy) ch;
+    for (final String key : ach.lookupTable.keySet()) {
+      if (!this.lookupTable.containsKey(key)) {
+        this.lookupTable.put(key, ach.lookupTable.get(key));
+      }
+    }
+
+    for (final Node n : ch.getNamespace().getChildren()) {
+      if (!this.namespace.contains(n.getFullName())) {
+        if (n instanceof NamedParameter) {
+          final NamedParameterNode np = (NamedParameterNode) n;
+          new NamedParameterNodeImpl<>(this.namespace, np.getName(), np.getFullName(), np.getFullArgName(),
+                  np.getSimpleArgName(), np.isSet(), np.isList(), np.getDocumentation(), np.getShortName(),
+                  np.getDefaultInstanceAsStrings());
+        } else if (n instanceof ClassNode) {
+          final ClassNode cn = (ClassNode) n;
+          new ClassNodeImpl<>(namespace, cn.getName(), cn.getFullName(), cn.isUnit(), cn.isInjectionCandidate(),
+                  cn.isExternalConstructor(), cn.getInjectableConstructors(), cn.getAllConstructors(),
+                  cn.getDefaultImplementation());
+        }
+      }
+    }
+    return this;
+  }
+
+  @Override
+  public Node getNamespace() {
+    return namespace;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchySerializer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchySerializer.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchySerializer.java
new file mode 100644
index 0000000..d0026ac
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/AvroClassHierarchySerializer.java
@@ -0,0 +1,263 @@
+/*
+ * 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.
+ */
+package org.apache.reef.tang.implementation.avro;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.io.*;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.reef.tang.ClassHierarchy;
+import org.apache.reef.tang.ClassHierarchySerializer;
+import org.apache.reef.tang.types.*;
+
+import javax.inject.Inject;
+import java.io.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Serialize and Deserialize ClassHierarchy to and from AvroClassHierarchy.
+ * This class is stateless and is therefore safe to reuse.
+ */
+public final class AvroClassHierarchySerializer implements ClassHierarchySerializer {
+
+  /**
+   * The Charset used for the JSON encoding.
+   */
+  private static final String JSON_CHARSET = "ISO-8859-1";
+
+  @Inject
+  public AvroClassHierarchySerializer() {
+  }
+
+  /**
+   * Serialize the ClassHierarchy into the AvroNode.
+   * @param ch ClassHierarchy to serialize
+   */
+  public AvroNode toAvro(final ClassHierarchy ch) {
+    return newAvroNode(ch.getNamespace());
+  }
+
+  /**
+   * Deserialize the ClassHierarchy from the AvroNode.
+   * @param n AvroNode to deserialize
+   */
+  public ClassHierarchy fromAvro(final AvroNode n) {
+    return new AvroClassHierarchy(n);
+  }
+
+  private AvroNode newAvroNode(final Node n) {
+    final List<AvroNode> children = new ArrayList<>();
+    for (final Node child : n.getChildren()) {
+      children.add(newAvroNode(child));
+    }
+    if (n instanceof ClassNode) {
+      final ClassNode<?> cn = (ClassNode<?>) n;
+      final ConstructorDef<?>[] injectable = cn.getInjectableConstructors();
+      final ConstructorDef<?>[] all = cn.getAllConstructors();
+      final List<ConstructorDef<?>> others = new ArrayList<>(Arrays.asList(all));
+      others.removeAll(Arrays.asList(injectable));
+
+      final List<AvroConstructorDef> injectableConstructors = new ArrayList<>();
+      for (final ConstructorDef<?> inj : injectable) {
+        injectableConstructors.add(newConstructorDef(inj));
+      }
+      final List<AvroConstructorDef> otherConstructors = new ArrayList<>();
+      for (final ConstructorDef<?> other : others) {
+        otherConstructors.add(newConstructorDef(other));
+      }
+      final List<CharSequence> implFullNames = new ArrayList<>();
+      for (final ClassNode<?> impl : cn.getKnownImplementations()) {
+        implFullNames.add(impl.getFullName());
+      }
+      return newClassNode(n.getName(), n.getFullName(), cn.isInjectionCandidate(), cn.isExternalConstructor(),
+              cn.isUnit(), injectableConstructors, otherConstructors, implFullNames, cn.getDefaultImplementation(),
+              children);
+    } else if (n instanceof NamedParameterNode) {
+      final NamedParameterNode<?> np = (NamedParameterNode<?>) n;
+      final List<CharSequence> defaultInstances = new ArrayList<>();
+      for (final CharSequence defaultInstance : np.getDefaultInstanceAsStrings()) {
+        defaultInstances.add(defaultInstance);
+      }
+      return newNamedParameterNode(np.getName(), np.getFullName(), np.getSimpleArgName(), np.getFullArgName(),
+              np.isSet(), np.isList(), np.getDocumentation(), np.getShortName(), defaultInstances, children);
+    } else if (n instanceof PackageNode) {
+      return newPackageNode(n.getName(), n.getFullName(), children);
+    } else {
+      throw new IllegalStateException("Encountered unknown type of Node: " + n);
+    }
+  }
+
+  private AvroNode newClassNode(final String name,
+                                       final String fullName,
+                                       final boolean isInjectionCandidate,
+                                       final boolean isExternalConstructor,
+                                       final boolean isUnit,
+                                       final List<AvroConstructorDef> injectableConstructors,
+                                       final List<AvroConstructorDef> otherConstructors,
+                                       final List<CharSequence> implFullNames,
+                                       final String defaultImplementation,
+                                       final List<AvroNode> children) {
+    return AvroNode.newBuilder()
+            .setName(name)
+            .setFullName(fullName)
+            .setClassNode(AvroClassNode.newBuilder()
+                    .setIsInjectionCandidate(isInjectionCandidate)
+                    .setIsExternalConstructor(isExternalConstructor)
+                    .setIsUnit(isUnit)
+                    .setInjectableConstructors(injectableConstructors)
+                    .setOtherConstructors(otherConstructors)
+                    .setImplFullNames(implFullNames)
+                    .setDefaultImplementation(defaultImplementation)
+                    .build())
+            .setChildren(children).build();
+  }
+
+  private AvroNode newNamedParameterNode(final String name,
+                                                final String fullName,
+                                                final String simpleArgClassName,
+                                                final String fullArgClassName,
+                                                final boolean isSet,
+                                                final boolean isList,
+                                                final String documentation,
+                                                final String shortName,
+                                                final List<CharSequence> instanceDefault,
+                                                final List<AvroNode> children) {
+
+    return AvroNode.newBuilder()
+            .setName(name)
+            .setFullName(fullName)
+            .setNamedParameterNode(AvroNamedParameterNode.newBuilder()
+                    .setSimpleArgClassName(simpleArgClassName)
+                    .setFullArgClassName(fullArgClassName)
+                    .setIsSet(isSet)
+                    .setIsList(isList)
+                    .setDocumentation(documentation)
+                    .setShortName(shortName)
+                    .setInstanceDefault(instanceDefault)
+                    .build())
+            .setChildren(children).build();
+  }
+
+  private AvroNode newPackageNode(final String name,
+                                         final String fullName,
+                                         final List<AvroNode> children) {
+    return AvroNode.newBuilder()
+            .setPackageNode(AvroPackageNode.newBuilder().build())
+            .setName(name).setFullName(fullName).setChildren(children).build();
+  }
+
+  private AvroConstructorArg newConstructorArg(final String fullArgClassName,
+                                                      final String namedParameterName,
+                                                      final boolean isFuture) {
+    return AvroConstructorArg.newBuilder()
+            .setFullArgClassName(fullArgClassName)
+            .setIsInjectionFuture(isFuture)
+            .setNamedParameterName(namedParameterName).build();
+  }
+
+  private AvroConstructorDef newConstructorDef(final ConstructorDef<?> def) {
+    final List<AvroConstructorArg> args = new ArrayList<>();
+    for (final ConstructorArg arg : def.getArgs()) {
+      args.add(newConstructorArg(arg.getType(), arg.getNamedParameterName(), arg.isInjectionFuture()));
+    }
+    return AvroConstructorDef.newBuilder()
+            .setFullClassName(def.getClassName())
+            .setConstructorArgs(args).build();
+  }
+
+  @Override
+  public void toFile(final ClassHierarchy classHierarchy, final File file) throws IOException {
+    final AvroNode avroNode = toAvro(classHierarchy);
+    final DatumWriter<AvroNode> avroNodeWriter = new SpecificDatumWriter<>(AvroNode.class);
+    try (final DataFileWriter<AvroNode> dataFileWriter = new DataFileWriter<>(avroNodeWriter)) {
+      dataFileWriter.create(avroNode.getSchema(), file);
+      dataFileWriter.append(avroNode);
+    }
+  }
+
+  @Override
+  public byte[] toByteArray(final ClassHierarchy classHierarchy) throws IOException {
+    final DatumWriter<AvroNode> requestWriter = new SpecificDatumWriter<>(AvroNode.class);
+    try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      final BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
+      requestWriter.write(toAvro(classHierarchy), encoder);
+      encoder.flush();
+      return out.toByteArray();
+    }
+  }
+
+  @Override
+  public String toString(final ClassHierarchy classHierarchy) throws IOException {
+    final DatumWriter<AvroNode> classHierarchyWriter = new SpecificDatumWriter<>(AvroNode.class);
+    try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      final JsonEncoder encoder = EncoderFactory.get().jsonEncoder(AvroNode.SCHEMA$, out);
+      classHierarchyWriter.write(toAvro(classHierarchy), encoder);
+      encoder.flush();
+      out.flush();
+      return out.toString(JSON_CHARSET);
+    }
+  }
+
+  @Override
+  public void toTextFile(ClassHierarchy classHierarchy, File file) throws IOException {
+    try (final Writer w = new FileWriter(file)) {
+      w.write(this.toString(classHierarchy));
+    }
+  }
+
+  @Override
+  public ClassHierarchy fromFile(final File file) throws IOException {
+    final AvroNode avroNode;
+    try (final DataFileReader<AvroNode> dataFileReader =
+                 new DataFileReader<>(file, new SpecificDatumReader<>(AvroNode.class))) {
+      avroNode = dataFileReader.next();
+    }
+    return fromAvro(avroNode);
+  }
+
+  @Override
+  public ClassHierarchy fromByteArray(final byte[] theBytes) throws IOException {
+    final BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(theBytes, null);
+    final SpecificDatumReader<AvroNode> reader = new SpecificDatumReader<>(AvroNode.class);
+    return fromAvro(reader.read(null, decoder));
+  }
+
+  @Override
+  public ClassHierarchy fromString(final String theString) throws IOException {
+    final JsonDecoder decoder = DecoderFactory.get().jsonDecoder(AvroNode.getClassSchema(), theString);
+    final SpecificDatumReader<AvroNode> reader = new SpecificDatumReader<>(AvroNode.class);
+    return fromAvro(reader.read(null, decoder));
+  }
+
+  @Override
+  public ClassHierarchy fromTextFile(final File file) throws IOException {
+    final StringBuilder stringBuilder = new StringBuilder();
+    try (final BufferedReader reader = new BufferedReader(new FileReader(file))) {
+      String line = reader.readLine();
+      while (line != null) {
+        stringBuilder.append(line);
+        line = reader.readLine();
+      }
+    }
+    return fromString(stringBuilder.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/package-info.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/package-info.java
new file mode 100644
index 0000000..7d568e1
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/avro/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * Implementation classes that translate between Tang's core API and Avro.
+ * This enables cross-language injection sessions.
+ */
+package org.apache.reef.tang.implementation.avro;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/protobuf/ProtocolBufferClassHierarchy.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/protobuf/ProtocolBufferClassHierarchy.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/protobuf/ProtocolBufferClassHierarchy.java
index 29b4939..e21eb4e 100644
--- a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/protobuf/ProtocolBufferClassHierarchy.java
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/implementation/protobuf/ProtocolBufferClassHierarchy.java
@@ -31,6 +31,10 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 
+/**
+ * @deprecated in 0.12. Use AvroClassHierarchy instead
+ */
+@Deprecated
 public class ProtocolBufferClassHierarchy implements ClassHierarchy {
 
   private static final String regex = "[\\.\\$\\+]";
@@ -47,6 +51,7 @@ public class ProtocolBufferClassHierarchy implements ClassHierarchy {
    * object is immutable, and does not make use of reflection to fill in any
    * missing values.  This allows it to represent non-native classes as well
    * as snapshots of Java class hierarchies.
+   * @deprecated in 0.12. Use AvroClassHierarchy instead
    */
   public ProtocolBufferClassHierarchy(ClassHierarchyProto.Node root) {
     namespace = new PackageNodeImpl();
@@ -197,6 +202,7 @@ public class ProtocolBufferClassHierarchy implements ClassHierarchy {
    *
    * @param classHierarchy
    * @return
+   * @deprecated in 0.12. Use AvroClassHierarchySerializer instead
    */
   public static ClassHierarchyProto.Node serialize(ClassHierarchy classHierarchy) {
     return serializeNode(classHierarchy.getNamespace());
@@ -208,6 +214,7 @@ public class ProtocolBufferClassHierarchy implements ClassHierarchy {
    * @param file
    * @param classHierarchy
    * @throws IOException
+   * @deprecated in 0.12. Use AvroClassHierarchySerializer instead
    */
   public static void serialize(final File file, final ClassHierarchy classHierarchy) throws IOException {
     final ClassHierarchyProto.Node node = serializeNode(classHierarchy.getNamespace());
@@ -224,6 +231,7 @@ public class ProtocolBufferClassHierarchy implements ClassHierarchy {
    * @param file
    * @return
    * @throws IOException
+   * @deprecated in 0.12. Use AvroClassHierarchySerializer instead
    */
   public static ClassHierarchy deserialize(final File file) throws IOException {
     try (final InputStream stream = new FileInputStream(file)) {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/util/ValidateConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/util/ValidateConfiguration.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/util/ValidateConfiguration.java
index 82c929c..31f4b96 100644
--- a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/util/ValidateConfiguration.java
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/util/ValidateConfiguration.java
@@ -36,6 +36,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 
+// TODO Clean up the code which are deprecated and commented out.
 public class ValidateConfiguration {
   private final String target;
   private final File ch;
@@ -85,6 +86,7 @@ public class ValidateConfiguration {
 
     final Tang t = Tang.Factory.getTang();
 
+    // TODO Use the AvroClassHierarchySerializer
     final ClassHierarchyProto.Node root;
     try (final InputStream chin = new FileInputStream(this.ch)) {
       root = ClassHierarchyProto.Node.parseFrom(chin);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/ClassHierarchyDeserializationTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/ClassHierarchyDeserializationTest.java b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/ClassHierarchyDeserializationTest.java
index a7e1469..af9648a 100644
--- a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/ClassHierarchyDeserializationTest.java
+++ b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/ClassHierarchyDeserializationTest.java
@@ -18,8 +18,8 @@
  */
 package org.apache.reef.tang;
 
-import org.apache.reef.tang.exceptions.BindException;
 import org.apache.reef.tang.exceptions.NameResolutionException;
+import org.apache.reef.tang.implementation.avro.AvroClassHierarchySerializer;
 import org.apache.reef.tang.formats.AvroConfigurationSerializer;
 import org.apache.reef.tang.formats.ConfigurationSerializer;
 import org.apache.reef.tang.implementation.protobuf.ProtocolBufferClassHierarchy;
@@ -35,16 +35,21 @@ import java.util.Set;
 
 /**
  * Test case for class hierarchy deserialization.
+ * TODO: The files should be created and deserialized by the AvroClassHierarchySerializer (REEF-400)
  */
 public class ClassHierarchyDeserializationTest {
+  private final ConfigurationSerializer configurationSerializer = new AvroConfigurationSerializer();
+  private final ClassHierarchySerializer classHierarchySerializer = new AvroClassHierarchySerializer();
 
   /**
    * generate task.bin from running .Net ClassHierarchyBuilder.exe
    */
   @Test
   public void testDeserializationForTasks() {
+    // TODO: The file should be written by Avro (REEF-400)
     try (final InputStream chin = Thread.currentThread().getContextClassLoader()
         .getResourceAsStream("Task.bin")) {
+      // TODO: Use AvroClassHierarchySerializer instead (REEF-400)
       final ClassHierarchyProto.Node root = ClassHierarchyProto.Node.parseFrom(chin);
       final ClassHierarchy ch = new ProtocolBufferClassHierarchy(root);
       Node n1 = ch.getNode("Org.Apache.REEF.Examples.Tasks.StreamingTasks.StreamTask1, Org.Apache.REEF.Examples.Tasks, Version=1.0.0.0, Culture=neutral, PublicKeyToken=null");
@@ -75,8 +80,10 @@ public class ClassHierarchyDeserializationTest {
     final ConfigurationBuilder taskConfigurationBuilder;
     final ConfigurationBuilder eventConfigurationBuilder;
 
+    // TODO: The file should be written by Avro (REEF-400)
     try (final InputStream chin = Thread.currentThread().getContextClassLoader()
         .getResourceAsStream("Task.bin")) {
+      // TODO: Use AvroClassHierarchySerializer instead (REEF-400)
       final ClassHierarchyProto.Node root = ClassHierarchyProto.Node.parseFrom(chin);
       final ClassHierarchy ch = new ProtocolBufferClassHierarchy(root);
       taskConfigurationBuilder = Tang.Factory.getTang().newConfigurationBuilder(ch);
@@ -85,8 +92,10 @@ public class ClassHierarchyDeserializationTest {
       throw new RuntimeException(message, e);
     }
 
+    // TODO: The file should be written by Avro (REEF-400)
     try (final InputStream chin = Thread.currentThread().getContextClassLoader()
         .getResourceAsStream("Event.bin")) {
+      // TODO: Use AvroClassHierarchySerializer instead (REEF-400)
       final ClassHierarchyProto.Node root = ClassHierarchyProto.Node.parseFrom(chin);
       final ClassHierarchy ch = new ProtocolBufferClassHierarchy(root);
       eventConfigurationBuilder = Tang.Factory.getTang().newConfigurationBuilder(ch);
@@ -103,8 +112,10 @@ public class ClassHierarchyDeserializationTest {
    */
   @Test
   public void testDeserializationForEvent() {
+    // TODO: The file should be written by Avro (REEF-400)
     try (final InputStream chin = Thread.currentThread().getContextClassLoader()
         .getResourceAsStream("Event.bin")) {
+      // TODO: Use AvroClassHierarchySerializer instead (REEF-400)
       final ClassHierarchyProto.Node root = ClassHierarchyProto.Node.parseFrom(chin);
       final ClassHierarchy ch = new ProtocolBufferClassHierarchy(root);
       final ConfigurationBuilder taskConfigurationBuilder = Tang.Factory.getTang()
@@ -120,15 +131,14 @@ public class ClassHierarchyDeserializationTest {
   public void testBindSetEntryWithSetOfT() throws IOException {
     final ClassHierarchy ns1 = Tang.Factory.getTang().getDefaultClassHierarchy();
     ns1.getNode(SetOfClasses.class.getName());
-    final ClassHierarchy ns2 = new ProtocolBufferClassHierarchy(ProtocolBufferClassHierarchy.serialize(ns1));
+    final ClassHierarchy ns2 = classHierarchySerializer.fromString(classHierarchySerializer.toString(ns1));
     final ConfigurationBuilder cb = Tang.Factory.getTang().newConfigurationBuilder(ns2);
 
     final NamedParameterNode<Set<Number>> n2 = (NamedParameterNode<Set<Number>>) ns1.getNode(SetOfClasses.class.getName());
     final Node fn = ns1.getNode(Float.class.getName());
     cb.bindSetEntry(n2, fn);
 
-    final ConfigurationSerializer serializer = new AvroConfigurationSerializer();
-    final Configuration c = serializer.fromString(serializer.toString(cb.build()), ns2);
+    final Configuration c = configurationSerializer.fromString(configurationSerializer.toString(cb.build()), ns2);
   }
 
   @Test
@@ -136,7 +146,7 @@ public class ClassHierarchyDeserializationTest {
   public void testBindSetEntryWithSetOfString() throws IOException {
     final ClassHierarchy ns1 = Tang.Factory.getTang().getDefaultClassHierarchy();
     ns1.getNode(SetOfStrings.class.getName());
-    final ClassHierarchy ns2 = new ProtocolBufferClassHierarchy(ProtocolBufferClassHierarchy.serialize(ns1));
+    final ClassHierarchy ns2 = classHierarchySerializer.fromString(classHierarchySerializer.toString(ns1));
     final ConfigurationBuilder cb = Tang.Factory.getTang().newConfigurationBuilder(ns2);
     cb.bindSetEntry(SetOfStrings.class.getName(), "four");
     cb.bindSetEntry(SetOfStrings.class.getName(), "five");
@@ -144,7 +154,6 @@ public class ClassHierarchyDeserializationTest {
     final NamedParameterNode<Set<String>> n2 = (NamedParameterNode<Set<String>>) ns1.getNode(SetOfStrings.class.getName());
     cb.bindSetEntry(n2, "six");
 
-    final ConfigurationSerializer serializer = new AvroConfigurationSerializer();
-    final Configuration c = serializer.fromString(serializer.toString(cb.build()), ns2);
+    final Configuration c = configurationSerializer.fromString(configurationSerializer.toString(cb.build()), ns2);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/TestClassHierarchy.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/TestClassHierarchy.java b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/TestClassHierarchy.java
index b0fea40..d5a5821 100644
--- a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/TestClassHierarchy.java
+++ b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/TestClassHierarchy.java
@@ -24,6 +24,7 @@ import org.apache.reef.tang.annotations.*;
 import org.apache.reef.tang.exceptions.ClassHierarchyException;
 import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.tang.exceptions.NameResolutionException;
+import org.apache.reef.tang.ClassHierarchySerializer;
 import org.apache.reef.tang.types.ClassNode;
 import org.apache.reef.tang.types.ConstructorDef;
 import org.apache.reef.tang.types.Node;
@@ -48,6 +49,7 @@ interface I1 {
 
 public class TestClassHierarchy {
   public ClassHierarchy ns;
+  public ClassHierarchySerializer serializer;
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
@@ -56,6 +58,7 @@ public class TestClassHierarchy {
   public void setUp() throws Exception {
     TangImpl.reset();
     ns = Tang.Factory.getTang().getDefaultClassHierarchy();
+    serializer = Tang.Factory.getTang().newInjector().getInstance(ClassHierarchySerializer.class);
   }
 
   public String s(Class<?> c) {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/avro/TestClassHierarchyRoundTrip.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/avro/TestClassHierarchyRoundTrip.java b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/avro/TestClassHierarchyRoundTrip.java
new file mode 100644
index 0000000..3b340f1
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/avro/TestClassHierarchyRoundTrip.java
@@ -0,0 +1,549 @@
+/*
+ * 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.
+ */
+package org.apache.reef.tang.implementation.avro;
+
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.exceptions.NameResolutionException;
+import org.apache.reef.tang.implementation.TangImpl;
+import org.apache.reef.tang.implementation.TestClassHierarchy;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+public class TestClassHierarchyRoundTrip extends TestClassHierarchy {
+  // The default ClassHierarchy
+  private void setup0() {
+    TangImpl.reset();
+    ns = Tang.Factory.getTang().getDefaultClassHierarchy();
+  }
+
+  // Serializes ClassHierarchy to file
+  private void setup1() {
+    TangImpl.reset();
+    try {
+      final File file = java.io.File.createTempFile("TangTest", "avroch");
+      serializer.toFile(ns, file);
+      ns = serializer.fromFile(file);
+      file.delete();
+    } catch (IOException e) {
+      Assert.fail(String.format("IOException when serialize/deserialize the ClassHierarchy", e));
+    }
+  }
+
+  // Serializes ClassHierarchy to TextFile
+  private void setup2() {
+    TangImpl.reset();
+    try {
+      final File textFile = java.io.File.createTempFile("TangTest", "avroch");
+      serializer.toTextFile(ns, textFile);
+      ns = serializer.fromTextFile(textFile);
+      textFile.delete();
+    } catch (IOException e) {
+      Assert.fail(String.format("IOException when serialize/deserialize the ClassHierarchy", e));
+    }
+  }
+
+  // Serializes ClassHierarchy to byte[]
+  private void setup3() {
+    TangImpl.reset();
+    try {
+      ns = serializer.fromByteArray(serializer.toByteArray(ns));
+    } catch (IOException e) {
+      Assert.fail(String.format("IOException when serialize/deserialize the ClassHierarchy", e));
+    }
+  }
+
+  // Serializes ClassHierarchy to String
+  private void setup4() {
+    TangImpl.reset();
+    try {
+      ns = serializer.fromString(serializer.toString(ns));
+    } catch (IOException e) {
+      Assert.fail(String.format("IOException when serialize/deserialize the ClassHierarchy", e));
+    }
+  }
+
+  @Test
+  @Override
+  public void testJavaString() throws NameResolutionException {
+    setup0();
+    super.testJavaString();
+    setup1();
+    super.testJavaString();
+    setup2();
+    super.testJavaString();
+    setup3();
+    super.testJavaString();
+    setup4();
+    super.testJavaString();
+  }
+
+  @Test
+  @Override
+  public void testSimpleConstructors() throws NameResolutionException {
+    setup0();
+    super.testSimpleConstructors();
+    setup1();
+    super.testSimpleConstructors();
+    setup2();
+    super.testSimpleConstructors();
+    setup3();
+    super.testSimpleConstructors();
+    setup4();
+    super.testSimpleConstructors();
+  }
+
+  @Test
+  @Override
+  public void testNamedParameterConstructors() throws NameResolutionException {
+    setup0();
+    super.testNamedParameterConstructors();
+    setup1();
+    super.testNamedParameterConstructors();
+    setup2();
+    super.testNamedParameterConstructors();
+    setup3();
+    super.testNamedParameterConstructors();
+    setup4();
+    super.testNamedParameterConstructors();
+  }
+
+  @Test
+  @Override
+  public void testArray() throws NameResolutionException {
+    setup0();
+    super.testArray();
+    setup1();
+    super.testArray();
+    setup2();
+    super.testArray();
+    setup3();
+    super.testArray();
+    setup4();
+    super.testArray();
+  }
+
+  @Test
+  @Override
+  public void testRepeatConstructorArg() throws NameResolutionException {
+    setup0();
+    super.testRepeatConstructorArg();
+    setup1();
+    super.testRepeatConstructorArg();
+    setup2();
+    super.testRepeatConstructorArg();
+    setup3();
+    super.testRepeatConstructorArg();
+    setup4();
+    super.testRepeatConstructorArg();
+  }
+
+  @Test
+  @Override
+  public void testRepeatConstructorArgClasses() throws NameResolutionException {
+    setup0();
+    super.testRepeatConstructorArgClasses();
+    setup1();
+    super.testRepeatConstructorArgClasses();
+    setup2();
+    super.testRepeatConstructorArgClasses();
+    setup3();
+    super.testRepeatConstructorArgClasses();
+    setup4();
+    super.testRepeatConstructorArgClasses();
+  }
+
+  @Test
+  @Override
+  public void testLeafRepeatedConstructorArgClasses() throws NameResolutionException {
+    setup0();
+    super.testLeafRepeatedConstructorArgClasses();
+    setup1();
+    super.testLeafRepeatedConstructorArgClasses();
+    setup2();
+    super.testLeafRepeatedConstructorArgClasses();
+    setup3();
+    super.testLeafRepeatedConstructorArgClasses();
+    setup4();
+    super.testLeafRepeatedConstructorArgClasses();
+  }
+
+  @Test
+  @Override
+  public void testNamedRepeatConstructorArgClasses() throws NameResolutionException {
+    setup0();
+    super.testNamedRepeatConstructorArgClasses();
+    setup1();
+    super.testNamedRepeatConstructorArgClasses();
+    setup2();
+    super.testNamedRepeatConstructorArgClasses();
+    setup3();
+    super.testNamedRepeatConstructorArgClasses();
+    setup4();
+    super.testNamedRepeatConstructorArgClasses();
+  }
+
+  @Test
+  @Override
+  public void testResolveDependencies() throws NameResolutionException {
+    setup0();
+    super.testResolveDependencies();
+    setup1();
+    super.testResolveDependencies();
+    setup2();
+    super.testResolveDependencies();
+    setup3();
+    super.testResolveDependencies();
+    setup4();
+    super.testResolveDependencies();
+  }
+
+  @Test
+  @Override
+  public void testDocumentedLocalNamedParameter() throws NameResolutionException {
+    setup0();
+    super.testDocumentedLocalNamedParameter();
+    setup1();
+    super.testDocumentedLocalNamedParameter();
+    setup2();
+    super.testDocumentedLocalNamedParameter();
+    setup3();
+    super.testDocumentedLocalNamedParameter();
+    setup4();
+    super.testDocumentedLocalNamedParameter();
+  }
+
+  @Test
+  @Override
+  public void testNamedParameterTypeMismatch() throws NameResolutionException {
+    setup0();
+    super.testNamedParameterTypeMismatch();
+    setup1();
+    super.testNamedParameterTypeMismatch();
+    setup2();
+    super.testNamedParameterTypeMismatch();
+    setup3();
+    super.testNamedParameterTypeMismatch();
+    setup4();
+    super.testNamedParameterTypeMismatch();
+  }
+
+  @Test
+  @Override
+  public void testUnannotatedName() throws NameResolutionException {
+    setup0();
+    super.testUnannotatedName();
+    setup1();
+    super.testUnannotatedName();
+    setup2();
+    super.testUnannotatedName();
+    setup3();
+    super.testUnannotatedName();
+    setup4();
+    super.testUnannotatedName();
+  }
+
+  @Test
+  @Override
+  public void testAnnotatedNotName() throws NameResolutionException {
+    setup0();
+    super.testAnnotatedNotName();
+    setup1();
+    super.testAnnotatedNotName();
+    setup2();
+    super.testAnnotatedNotName();
+    setup3();
+    super.testAnnotatedNotName();
+    setup4();
+    super.testAnnotatedNotName();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture1() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture1();
+    setup1();
+    super.testGenericTorture1();
+    setup2();
+    super.testGenericTorture1();
+    setup3();
+    super.testGenericTorture1();
+    setup4();
+    super.testGenericTorture1();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture3() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture3();
+    setup1();
+    super.testGenericTorture3();
+    setup2();
+    super.testGenericTorture3();
+    setup3();
+    super.testGenericTorture3();
+    setup4();
+    super.testGenericTorture3();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture4() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture4();
+    setup1();
+    super.testGenericTorture4();
+    setup2();
+    super.testGenericTorture4();
+    setup3();
+    super.testGenericTorture4();
+    setup4();
+    super.testGenericTorture4();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture5() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture5();
+    setup1();
+    super.testGenericTorture5();
+    setup2();
+    super.testGenericTorture5();
+    setup3();
+    super.testGenericTorture5();
+    setup4();
+    super.testGenericTorture5();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture6() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture6();
+    setup1();
+    super.testGenericTorture6();
+    setup2();
+    super.testGenericTorture6();
+    setup3();
+    super.testGenericTorture6();
+    setup4();
+    super.testGenericTorture6();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture7() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture7();
+    setup1();
+    super.testGenericTorture7();
+    setup2();
+    super.testGenericTorture7();
+    setup3();
+    super.testGenericTorture7();
+    setup4();
+    super.testGenericTorture7();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture8() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture8();
+    setup1();
+    super.testGenericTorture8();
+    setup2();
+    super.testGenericTorture8();
+    setup3();
+    super.testGenericTorture8();
+    setup4();
+    super.testGenericTorture8();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture9() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture9();
+    setup1();
+    super.testGenericTorture9();
+    setup2();
+    super.testGenericTorture9();
+    setup3();
+    super.testGenericTorture9();
+    setup4();
+    super.testGenericTorture9();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture10() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture10();
+    setup1();
+    super.testGenericTorture10();
+    setup2();
+    super.testGenericTorture10();
+    setup3();
+    super.testGenericTorture10();
+    setup4();
+    super.testGenericTorture10();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture11() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture11();
+    setup1();
+    super.testGenericTorture11();
+    setup2();
+    super.testGenericTorture11();
+    setup3();
+    super.testGenericTorture11();
+    setup4();
+    super.testGenericTorture11();
+  }
+
+  @Test
+  @Override
+  public void testGenericTorture12() throws NameResolutionException {
+    setup0();
+    super.testGenericTorture12();
+    setup1();
+    super.testGenericTorture12();
+    setup2();
+    super.testGenericTorture12();
+    setup3();
+    super.testGenericTorture12();
+    setup4();
+    super.testGenericTorture12();
+  }
+
+  @Test
+  @Override
+  public void testInjectNonStaticLocalArgClass() throws NameResolutionException {
+    setup0();
+    super.testInjectNonStaticLocalArgClass();
+    setup1();
+    super.testInjectNonStaticLocalArgClass();
+    setup2();
+    super.testInjectNonStaticLocalArgClass();
+    setup3();
+    super.testInjectNonStaticLocalArgClass();
+    setup4();
+    super.testInjectNonStaticLocalArgClass();
+  }
+
+  @Test
+  @Override
+  public void testOKShortNames() throws NameResolutionException {
+    setup0();
+    super.testOKShortNames();
+    setup1();
+    super.testOKShortNames();
+    setup2();
+    super.testOKShortNames();
+    setup3();
+    super.testOKShortNames();
+    setup4();
+    super.testOKShortNames();
+  }
+
+  @Test
+  @Override
+  public void testRoundTripInnerClassNames() throws NameResolutionException, ClassNotFoundException {
+    setup0();
+    super.testRoundTripInnerClassNames();
+    setup1();
+    super.testRoundTripInnerClassNames();
+    setup2();
+    super.testRoundTripInnerClassNames();
+    setup3();
+    super.testRoundTripInnerClassNames();
+    setup4();
+    super.testRoundTripInnerClassNames();
+  }
+
+  @Test
+  @Override
+  public void testUnitIsInjectable() throws NameResolutionException, InjectionException {
+    setup0();
+    super.testUnitIsInjectable();
+    setup1();
+    super.testUnitIsInjectable();
+    setup2();
+    super.testUnitIsInjectable();
+    setup3();
+    super.testUnitIsInjectable();
+    setup4();
+    super.testUnitIsInjectable();
+  }
+
+  @Test
+  @Override
+  public void testBadUnitDecl() throws NameResolutionException {
+    setup0();
+    super.testBadUnitDecl();
+    setup1();
+    super.testBadUnitDecl();
+    setup2();
+    super.testBadUnitDecl();
+    setup3();
+    super.testBadUnitDecl();
+    setup4();
+    super.testBadUnitDecl();
+  }
+
+  @Test
+  @Override
+  public void nameCantBindWrongSubclassAsDefault() throws NameResolutionException {
+    setup0();
+    super.nameCantBindWrongSubclassAsDefault();
+    setup1();
+    super.nameCantBindWrongSubclassAsDefault();
+    setup2();
+    super.nameCantBindWrongSubclassAsDefault();
+    setup3();
+    super.nameCantBindWrongSubclassAsDefault();
+    setup4();
+    super.nameCantBindWrongSubclassAsDefault();
+  }
+
+  @Test
+  @Override
+  public void ifaceCantBindWrongImplAsDefault() throws NameResolutionException {
+    setup0();
+    super.ifaceCantBindWrongImplAsDefault();
+    setup1();
+    super.ifaceCantBindWrongImplAsDefault();
+    setup2();
+    super.ifaceCantBindWrongImplAsDefault();
+    setup3();
+    super.ifaceCantBindWrongImplAsDefault();
+    setup4();
+    super.ifaceCantBindWrongImplAsDefault();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/protobuf/TestClassHierarchyRoundTrip.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/protobuf/TestClassHierarchyRoundTrip.java b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/protobuf/TestClassHierarchyRoundTrip.java
deleted file mode 100644
index d50ded5..0000000
--- a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/implementation/protobuf/TestClassHierarchyRoundTrip.java
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * 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.
- */
-package org.apache.reef.tang.implementation.protobuf;
-
-import org.apache.reef.tang.Tang;
-import org.apache.reef.tang.exceptions.InjectionException;
-import org.apache.reef.tang.exceptions.NameResolutionException;
-import org.apache.reef.tang.implementation.TangImpl;
-import org.apache.reef.tang.implementation.TestClassHierarchy;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-
-public class TestClassHierarchyRoundTrip extends TestClassHierarchy {
-
-  private void setup1() {
-    TangImpl.reset();
-    ns = Tang.Factory.getTang().getDefaultClassHierarchy();
-  }
-
-  private void setup2() {
-    TangImpl.reset();
-    ns = new ProtocolBufferClassHierarchy(ProtocolBufferClassHierarchy.serialize(ns));
-  }
-
-  private void setup3() {
-    TangImpl.reset();
-    try {
-      final File file = java.io.File.createTempFile("testProto", "bin");
-      ProtocolBufferClassHierarchy.serialize(file, ns);
-      ns = ProtocolBufferClassHierarchy.deserialize(file);
-      file.delete();
-    } catch (IOException e) {
-      Assert.fail(String.format("IOException when serialize/deserialize proto buffer file ", e));
-    }
-  }
-
-  @Test
-  @Override
-  public void testJavaString() throws NameResolutionException {
-    setup1();
-    super.testJavaString();
-    setup2();
-    super.testJavaString();
-    setup3();
-    super.testJavaString();
-  }
-
-  @Test
-  @Override
-  public void testSimpleConstructors() throws NameResolutionException {
-    setup1();
-    super.testSimpleConstructors();
-    setup2();
-    super.testSimpleConstructors();
-    setup3();
-    super.testSimpleConstructors();
-  }
-
-  @Test
-  @Override
-  public void testNamedParameterConstructors() throws NameResolutionException {
-    setup1();
-    super.testNamedParameterConstructors();
-    setup2();
-    super.testNamedParameterConstructors();
-    setup3();
-    super.testNamedParameterConstructors();
-  }
-
-  @Test
-  @Override
-  public void testArray() throws NameResolutionException {
-    setup1();
-    super.testArray();
-    setup2();
-    super.testArray();
-    setup3();
-    super.testArray();
-  }
-
-  @Test
-  @Override
-  public void testRepeatConstructorArg() throws NameResolutionException {
-    setup1();
-    super.testRepeatConstructorArg();
-    setup2();
-    super.testRepeatConstructorArg();
-    setup3();
-    super.testRepeatConstructorArg();
-  }
-
-  @Test
-  @Override
-  public void testRepeatConstructorArgClasses() throws NameResolutionException {
-    setup1();
-    super.testRepeatConstructorArgClasses();
-    setup2();
-    super.testRepeatConstructorArgClasses();
-    setup3();
-    super.testRepeatConstructorArgClasses();
-  }
-
-  @Test
-  @Override
-  public void testLeafRepeatedConstructorArgClasses() throws NameResolutionException {
-    setup1();
-    super.testLeafRepeatedConstructorArgClasses();
-    setup2();
-    super.testLeafRepeatedConstructorArgClasses();
-    setup3();
-    super.testLeafRepeatedConstructorArgClasses();
-  }
-
-  @Test
-  @Override
-  public void testNamedRepeatConstructorArgClasses() throws NameResolutionException {
-    setup1();
-    super.testNamedRepeatConstructorArgClasses();
-    setup2();
-    super.testNamedRepeatConstructorArgClasses();
-    setup3();
-    super.testNamedRepeatConstructorArgClasses();
-  }
-
-  @Test
-  @Override
-  public void testResolveDependencies() throws NameResolutionException {
-    setup1();
-    super.testResolveDependencies();
-    setup2();
-    super.testResolveDependencies();
-    setup3();
-    super.testResolveDependencies();
-  }
-
-  @Test
-  @Override
-  public void testDocumentedLocalNamedParameter() throws NameResolutionException {
-    setup1();
-    super.testDocumentedLocalNamedParameter();
-    setup2();
-    super.testDocumentedLocalNamedParameter();
-    setup3();
-    super.testDocumentedLocalNamedParameter();
-  }
-
-  @Test
-  @Override
-  public void testNamedParameterTypeMismatch() throws NameResolutionException {
-    setup1();
-    super.testNamedParameterTypeMismatch();
-    setup2();
-    super.testNamedParameterTypeMismatch();
-    setup3();
-    super.testNamedParameterTypeMismatch();
-  }
-
-  @Test
-  @Override
-  public void testUnannotatedName() throws NameResolutionException {
-    setup1();
-    super.testUnannotatedName();
-    setup2();
-    super.testUnannotatedName();
-    setup3();
-    super.testUnannotatedName();
-  }
-
-  @Test
-  @Override
-  public void testAnnotatedNotName() throws NameResolutionException {
-    setup1();
-    super.testAnnotatedNotName();
-    setup2();
-    super.testAnnotatedNotName();
-    setup3();
-    super.testAnnotatedNotName();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture1() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture1();
-    setup2();
-    super.testGenericTorture1();
-    setup3();
-    super.testGenericTorture1();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture2() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture2();
-    setup2();
-    super.testGenericTorture2();
-    setup3();
-    super.testGenericTorture2();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture3() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture3();
-    setup2();
-    super.testGenericTorture3();
-    setup3();
-    super.testGenericTorture3();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture4() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture4();
-    setup2();
-    super.testGenericTorture4();
-    setup3();
-    super.testGenericTorture4();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture5() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture5();
-    setup2();
-    super.testGenericTorture5();
-    setup3();
-    super.testGenericTorture5();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture6() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture6();
-    setup2();
-    super.testGenericTorture6();
-    setup3();
-    super.testGenericTorture6();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture7() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture7();
-    setup2();
-    super.testGenericTorture7();
-    setup3();
-    super.testGenericTorture7();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture8() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture8();
-    setup2();
-    super.testGenericTorture8();
-    setup3();
-    super.testGenericTorture8();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture9() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture9();
-    setup2();
-    super.testGenericTorture9();
-    setup3();
-    super.testGenericTorture9();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture10() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture10();
-    setup2();
-    super.testGenericTorture10();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture11() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture11();
-    setup2();
-    super.testGenericTorture11();
-  }
-
-  @Test
-  @Override
-  public void testGenericTorture12() throws NameResolutionException {
-    setup1();
-    super.testGenericTorture12();
-    setup2();
-    super.testGenericTorture12();
-  }
-
-  @Test
-  @Override
-  public void testInjectNonStaticLocalArgClass() throws NameResolutionException {
-    setup1();
-    super.testInjectNonStaticLocalArgClass();
-    setup2();
-    super.testInjectNonStaticLocalArgClass();
-    setup3();
-    super.testInjectNonStaticLocalArgClass();
-  }
-
-  @Test
-  @Override
-  public void testOKShortNames() throws NameResolutionException {
-    setup1();
-    super.testOKShortNames();
-    setup2();
-    super.testOKShortNames();
-    setup3();
-    super.testOKShortNames();
-  }
-
-  @Test
-  @Override
-  public void testRoundTripInnerClassNames() throws NameResolutionException, ClassNotFoundException {
-    setup1();
-    super.testRoundTripInnerClassNames();
-    setup2();
-    super.testRoundTripInnerClassNames();
-    setup3();
-    super.testRoundTripInnerClassNames();
-  }
-
-  @Test
-  @Override
-  public void testUnitIsInjectable() throws NameResolutionException, InjectionException {
-    setup1();
-    super.testUnitIsInjectable();
-    setup2();
-    super.testUnitIsInjectable();
-    setup3();
-    super.testUnitIsInjectable();
-  }
-
-  @Test
-  @Override
-  public void testBadUnitDecl() throws NameResolutionException {
-    setup1();
-    super.testBadUnitDecl();
-    setup2();
-    super.testBadUnitDecl();
-    setup3();
-    super.testBadUnitDecl();
-  }
-
-  @Test
-  @Override
-  public void nameCantBindWrongSubclassAsDefault() throws NameResolutionException {
-    setup1();
-    super.nameCantBindWrongSubclassAsDefault();
-    setup2();
-    super.nameCantBindWrongSubclassAsDefault();
-    setup3();
-    super.nameCantBindWrongSubclassAsDefault();
-  }
-
-  @Test
-  @Override
-  public void ifaceCantBindWrongImplAsDefault() throws NameResolutionException {
-    setup1();
-    super.ifaceCantBindWrongImplAsDefault();
-    setup2();
-    super.ifaceCantBindWrongImplAsDefault();
-    setup3();
-    super.ifaceCantBindWrongImplAsDefault();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/96f9e94f/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/test/RoundTripTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/test/RoundTripTest.java b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/test/RoundTripTest.java
index 2dd34a6..275ec0a 100644
--- a/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/test/RoundTripTest.java
+++ b/lang/java/reef-tang/tang/src/test/java/org/apache/reef/tang/test/RoundTripTest.java
@@ -21,7 +21,7 @@ package org.apache.reef.tang.test;
 import org.apache.reef.tang.ClassHierarchy;
 import org.apache.reef.tang.Configuration;
 import org.apache.reef.tang.Tang;
-import org.apache.reef.tang.implementation.protobuf.ProtocolBufferClassHierarchy;
+import org.apache.reef.tang.implementation.avro.AvroClassHierarchySerializer;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -48,7 +48,8 @@ public abstract class RoundTripTest {
   public void testRoundTripWithClassHierarchy() throws Exception {
     // TODO: use 'getConfiguration' instead of 'getConfigurationWithoutList' after #192 is fixed
     final Configuration confBefore = ObjectTreeTest.getConfigurationWithoutList();
-    final ClassHierarchy c = new ProtocolBufferClassHierarchy(ProtocolBufferClassHierarchy.serialize(confBefore.getClassHierarchy()));
+    final AvroClassHierarchySerializer chSerializer = new AvroClassHierarchySerializer();
+    final ClassHierarchy c = chSerializer.fromAvro(chSerializer.toAvro(confBefore.getClassHierarchy()));
     final Configuration confAfter = roundTrip(confBefore, c);
     Assert.assertEquals(confBefore.getNamedParameters().size(), confAfter.getNamedParameters().size());
     //For now, we cannot use ProtocolBufferClassHierarchy to do injection