You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by th...@apache.org on 2017/05/19 17:12:35 UTC

[21/36] avro git commit: [add] resolution code using new reusable visitor.

[add] resolution code using new reusable visitor.

Project: http://git-wip-us.apache.org/repos/asf/avro/repo
Commit: http://git-wip-us.apache.org/repos/asf/avro/commit/874fff78
Tree: http://git-wip-us.apache.org/repos/asf/avro/tree/874fff78
Diff: http://git-wip-us.apache.org/repos/asf/avro/diff/874fff78

Branch: refs/heads/master
Commit: 874fff780dfe6d6ee8a3fa13f461110794b56f35
Parents: 884fbab
Author: Zoltan Farkas <zo...@yahoo.com>
Authored: Thu Apr 6 13:39:48 2017 -0400
Committer: Zoltan Farkas <zo...@yahoo.com>
Committed: Thu Apr 6 13:39:48 2017 -0400

----------------------------------------------------------------------
 .../avro/compiler/idl/ResolvingVisitor.java     | 166 +++++++++++++++
 .../avro/compiler/idl/SchemaResolver.java       | 203 +++++--------------
 .../avro/compiler/schema/CloningVisitor.java    | 155 ++++++++++++++
 lang/java/compiler/src/test/idl/cycle.avdl      |  24 +++
 .../avro/compiler/idl/SchemaResolverTest.java   |  44 ++++
 .../avro/compiler/schema/SchemasTest.java       |  87 ++++++++
 6 files changed, 525 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/avro/blob/874fff78/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java b/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java
new file mode 100644
index 0000000..e201598
--- /dev/null
+++ b/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java
@@ -0,0 +1,166 @@
+
+package org.apache.avro.compiler.idl;
+
+import avro.shaded.com.google.common.base.Function;
+import java.util.ArrayList;
+import java.util.IdentityHashMap;
+import java.util.List;
+import org.apache.avro.AvroTypeException;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.compiler.schema.SchemaVisitor;
+import org.apache.avro.compiler.schema.SchemaVisitorAction;
+import org.apache.avro.compiler.schema.Schemas;
+
+/**
+ * this visitor will create a clone of the original Schema and will also resolve all unresolved schemas
+ *
+ * by default. what attributes are copied is customizable.
+ * @author zoly
+ */
+public final class ResolvingVisitor implements SchemaVisitor<Schema> {
+
+  private final IdentityHashMap<Schema, Schema> replace;
+  private final Function<String, Schema> symbolTable;
+
+  private final Schema root;
+
+
+  public ResolvingVisitor(final Schema root, final IdentityHashMap<Schema, Schema> replace,
+          final Function<String, Schema> symbolTable) {
+    this.replace = replace;
+    this.symbolTable = symbolTable;
+    this.root = root;
+  }
+
+  @Override
+  public SchemaVisitorAction visitTerminal(final Schema terminal) {
+    Schema.Type type = terminal.getType();
+    Schema newSchema;
+    switch (type) {
+      case RECORD: // recursion.
+      case ARRAY:
+      case MAP:
+      case UNION:
+        if (!replace.containsKey(terminal)) {
+          throw new IllegalStateException("Schema " + terminal + " must be already processed");
+        }
+        return SchemaVisitorAction.CONTINUE;
+      case BOOLEAN:
+      case BYTES:
+      case DOUBLE:
+      case FLOAT:
+      case INT:
+      case LONG:
+      case NULL:
+      case STRING:
+        newSchema = Schema.create(type);
+      break;
+      case ENUM:
+        newSchema = Schema.createEnum(terminal.getName(), terminal.getDoc(),
+                terminal.getNamespace(), terminal.getEnumSymbols());
+        break;
+      case FIXED:
+        newSchema = Schema.createFixed(terminal.getName(), terminal.getDoc(),
+                terminal.getNamespace(), terminal.getFixedSize());
+        break;
+      default:
+        throw new IllegalStateException("Unsupported schema " + terminal);
+    }
+    copyAllProperties(terminal, newSchema);
+    replace.put(terminal, newSchema);
+    return SchemaVisitorAction.CONTINUE;
+  }
+
+  public static void copyAllProperties(final Schema first, final Schema second) {
+    Schemas.copyLogicalTypes(first, second);
+    Schemas.copyAliases(first, second);
+    Schemas.copyProperties(first, second);
+  }
+
+  public static void copyAllProperties(final Field first, final Field second) {
+    Schemas.copyAliases(first, second);
+    Schemas.copyProperties(first, second);
+  }
+
+  @Override
+  public SchemaVisitorAction visitNonTerminal(final Schema nt) {
+    Schema.Type type = nt.getType();
+    if  (type == Schema.Type.RECORD) {
+        if (SchemaResolver.isUnresolvedSchema(nt)) {
+          // unresolved schema will get a replacement that we already encountered,
+          // or we will attempt to resolve.
+          final String unresolvedSchemaName = SchemaResolver.getUnresolvedSchemaName(nt);
+          Schema resSchema = symbolTable.apply(unresolvedSchemaName);
+          if (resSchema == null) {
+            throw new AvroTypeException("Unable to resolve " + unresolvedSchemaName);
+          }
+          Schema replacement = replace.get(resSchema);
+          if (replacement == null) {
+            replace.put(nt, Schemas.visit(resSchema, new ResolvingVisitor(resSchema,
+                    new IdentityHashMap<Schema, Schema>(), symbolTable)));
+          } else {
+            replace.put(nt, replacement);
+          }
+        } else {
+          // create a fieldless clone. Fields will be added in afterVisitNonTerminal.
+          Schema newSchema = Schema.createRecord(nt.getName(), nt.getDoc(), nt.getNamespace(), nt.isError());
+          copyAllProperties(nt, newSchema);
+          replace.put(nt, newSchema);
+        }
+    }
+    return SchemaVisitorAction.CONTINUE;
+  }
+
+  @Override
+  public SchemaVisitorAction afterVisitNonTerminal(final Schema nt) {
+     Schema.Type type = nt.getType();
+     Schema newSchema;
+     switch (type) {
+       case RECORD:
+         if (!SchemaResolver.isUnresolvedSchema(nt)) {
+            newSchema = replace.get(nt);
+            List<Schema.Field> fields = nt.getFields();
+            List<Schema.Field> newFields = new ArrayList<Schema.Field>(fields.size());
+            for (Schema.Field field : fields) {
+             Schema.Field newField = new Schema.Field(field.name(), replace.get(field.schema()),
+                     field.doc(), field.defaultVal(), field.order());
+             copyAllProperties(field, newField);
+             newFields.add(newField);
+            }
+            newSchema.setFields(newFields);
+         }
+         return SchemaVisitorAction.CONTINUE;
+       case UNION:
+          List<Schema> types = nt.getTypes();
+          List<Schema> newTypes = new ArrayList<Schema>(types.size());
+          for (Schema sch : types) {
+            newTypes.add(replace.get(sch));
+          }
+          newSchema = Schema.createUnion(newTypes);
+          break;
+       case ARRAY:
+         newSchema = Schema.createArray(replace.get(nt.getElementType()));
+         break;
+       case MAP:
+         newSchema = Schema.createMap(replace.get(nt.getValueType()));
+         break;
+       default:
+         throw new IllegalStateException("Illegal type " + type + ", schema " + nt);
+     }
+     copyAllProperties(nt, newSchema);
+     replace.put(nt, newSchema);
+     return SchemaVisitorAction.CONTINUE;
+  }
+
+  @Override
+  public Schema get() {
+    return replace.get(root);
+  }
+
+  @Override
+  public String toString() {
+    return "ResolvingVisitor{" + "replace=" + replace + ", symbolTable=" + symbolTable + ", root=" + root + '}';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/avro/blob/874fff78/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/SchemaResolver.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/SchemaResolver.java b/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/SchemaResolver.java
index 6aed787..f456a18 100644
--- a/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/SchemaResolver.java
+++ b/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/SchemaResolver.java
@@ -15,10 +15,11 @@
  */
 package org.apache.avro.compiler.idl;
 
+import avro.shaded.com.google.common.base.Function;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.avro.Protocol;
@@ -28,7 +29,6 @@ import org.apache.avro.compiler.schema.Schemas;
 /**
  * Utility class to resolve schemas that are unavailable at the time they are referenced in the IDL.
  */
-
 final class SchemaResolver {
 
   private SchemaResolver() {
@@ -40,21 +40,36 @@ final class SchemaResolver {
 
   private static final String UR_SCHEMA_NS = "org.apache.avro.compiler";
 
+  /**
+   * Create a schema to represent a "unresolved" schema.
+   * (used to represent a schema where the definition is not known at the time)
+   * This concept might be generalizable...
+   * @param name
+   * @return
+   */
   static Schema unresolvedSchema(final String name) {
-
-
     Schema schema = Schema.createRecord(UR_SCHEMA_NAME, "unresolved schema",
             UR_SCHEMA_NS, false, Collections.EMPTY_LIST);
     schema.addProp(UR_SCHEMA_ATTR, name);
     return schema;
   }
 
+  /**
+   * Is this a unresolved schema.
+   * @param schema
+   * @return
+   */
   static boolean isUnresolvedSchema(final Schema schema) {
     return (schema.getType() == Schema.Type.RECORD && schema.getProp(UR_SCHEMA_ATTR) != null
             && UR_SCHEMA_NAME.equals(schema.getName())
             && UR_SCHEMA_NS.equals(schema.getNamespace()));
   }
 
+  /**
+   * get the unresolved schema name.
+   * @param schema
+   * @return
+   */
   static String getUnresolvedSchemaName(final Schema schema) {
     if (!isUnresolvedSchema(schema)) {
       throw new IllegalArgumentException("Not a unresolved schema: " + schema);
@@ -68,34 +83,35 @@ final class SchemaResolver {
   }
 
   /**
-   * Resolve all unresolved schema references from a protocol.
-   * @param protocol - the protocol with unresolved schema references.
-   * @return - a new protocol instance based on the provided protocol with all unresolved schema references resolved.
+   * Will clone the provided protocol while resolving all unreferenced schemas
+   * @param protocol
+   * @return
    */
   static Protocol resolve(final Protocol protocol) {
     Protocol result = new Protocol(protocol.getName(), protocol.getDoc(), protocol.getNamespace());
     final Collection<Schema> types = protocol.getTypes();
+    // replace unresolved schemas.
     List<Schema> newSchemas = new ArrayList(types.size());
-    Map<String, Schema> resolved = new HashMap<String, Schema>();
+    IdentityHashMap<Schema, Schema> replacements = new IdentityHashMap<Schema, Schema>();
     for (Schema schema : types) {
-      newSchemas.add(resolve(schema, protocol, resolved));
+      newSchemas.add(Schemas.visit(schema, new ResolvingVisitor(schema, replacements, new SymbolTable(protocol))));
     }
     result.setTypes(newSchemas); // replace types with resolved ones
 
+    // Resolve all schemas refferenced by protocol Messages.
     for (Map.Entry<String, Protocol.Message> entry : protocol.getMessages().entrySet()) {
       Protocol.Message value = entry.getValue();
       Protocol.Message nvalue;
       if (value.isOneWay()) {
-        Schema request = value.getRequest();
+        Schema replacement = resolve(replacements, value.getRequest(), protocol);
         nvalue = result.createMessage(value.getName(), value.getDoc(),
-                value.getObjectProps(), getResolvedSchema(request, resolved));
+                value.getObjectProps(), replacement);
       } else {
-        Schema request = value.getRequest();
-        Schema response = value.getResponse();
-        Schema errors = value.getErrors();
+        Schema request = resolve(replacements, value.getRequest(), protocol);
+        Schema response = resolve(replacements, value.getResponse(), protocol);
+        Schema errors = resolve(replacements, value.getErrors(), protocol);
         nvalue = result.createMessage(value.getName(), value.getDoc(),
-                value.getObjectProps(), getResolvedSchema(request, resolved),
-                getResolvedSchema(response, resolved), getResolvedSchema(errors, resolved));
+                value.getObjectProps(), request, response, errors);
       }
       result.getMessages().put(entry.getKey(), nvalue);
     }
@@ -103,148 +119,27 @@ final class SchemaResolver {
     return result;
   }
 
-
-  /**
-   * Resolve all unresolved schema references.
-   * @param schema - the schema to resolved references for.
-   * @param protocol - the protocol we resolve the schema's for.
-   * (we lookup all unresolved schema references in the protocol)
-   * @param resolved - a map of all resolved schema's so far.
-   * @return - a instance of the resolved schema.
-   */
-  static Schema resolve(final Schema schema, final Protocol protocol, final Map<String, Schema> resolved) {
-    final String fullName = schema.getFullName();
-    if (fullName != null && resolved.containsKey(fullName)) {
-      return resolved.get(schema.getFullName());
-    } else if (isUnresolvedSchema(schema)) {
-      final String unresolvedSchemaName = getUnresolvedSchemaName(schema);
-      Schema type = protocol.getType(unresolvedSchemaName);
-      if (type == null) {
-        throw new IllegalArgumentException("Cannot resolve " + unresolvedSchemaName);
-      }
-      return resolve(type, protocol, resolved);
-    } else {
-      switch (schema.getType()) {
-        case RECORD:
-          Schema createRecord = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(),
-                  schema.isError());
-          resolved.put(schema.getFullName(), createRecord);
-          final List<Schema.Field> currFields = schema.getFields();
-          List<Schema.Field> newFields = new ArrayList<Schema.Field>(currFields.size());
-          for (Schema.Field field : currFields) {
-            if (field.name().equals("hash")) {
-              System.err.println(field);
-            }
-            Schema.Field nf = new Schema.Field(field.name(), resolve(field.schema(), protocol, resolved),
-                    field.doc(), field.defaultVal(), field.order());
-            Schemas.copyAliases(field, nf);
-            Schemas.copyProperties(field, nf);
-            newFields.add(nf);
-          }
-          createRecord.setFields(newFields);
-          Schemas.copyLogicalTypes(schema, createRecord);
-          Schemas.copyProperties(schema, createRecord);
-          return createRecord;
-        case MAP:
-          Schema result = Schema.createMap(resolve(schema.getValueType(), protocol, resolved));
-          Schemas.copyProperties(schema, result);
-          return result;
-        case ARRAY:
-          Schema aresult = Schema.createArray(resolve(schema.getElementType(), protocol, resolved));
-          Schemas.copyProperties(schema, aresult);
-          return aresult;
-        case UNION:
-          final List<Schema> uTypes = schema.getTypes();
-          List<Schema> newTypes = new ArrayList<Schema>(uTypes.size());
-          for (Schema s : uTypes) {
-            newTypes.add(resolve(s, protocol, resolved));
-          }
-          Schema bresult = Schema.createUnion(newTypes);
-          Schemas.copyProperties(schema, bresult);
-          return bresult;
-        case ENUM:
-        case FIXED:
-        case STRING:
-        case BYTES:
-        case INT:
-        case LONG:
-        case FLOAT:
-        case DOUBLE:
-        case BOOLEAN:
-        case NULL:
-          return schema;
-        default:
-          throw new RuntimeException("Unknown type: " + schema);
-      }
+  private static Schema resolve(final IdentityHashMap<Schema, Schema> replacements,
+          final Schema request, final Protocol protocol) {
+    Schema replacement = replacements.get(request);
+    if (replacement == null) {
+      replacement = Schemas.visit(request, new ResolvingVisitor(request, replacements,
+              new SymbolTable(protocol)));
     }
+    return replacement;
   }
 
-  /**
-   * get the resolved schema.
-   * @param schema - the schema we want to get the resolved equivalent for.
-   * @param resolved - a Map wil all resolved schemas
-   * @return - the resolved schema.
-   */
-  public static Schema getResolvedSchema(final Schema schema, final Map<String, Schema> resolved) {
-    if (schema == null) {
-      return null;
+  private static class SymbolTable implements Function<String, Schema> {
+
+    private final Protocol symbolTable;
+
+    public SymbolTable(Protocol symbolTable) {
+      this.symbolTable = symbolTable;
     }
-    final String fullName = schema.getFullName();
-    if (fullName != null && resolved.containsKey(fullName)) {
-      return resolved.get(schema.getFullName());
-    } else {
-      switch (schema.getType()) {
-        case RECORD:
-          Schema createRecord = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(),
-              schema.isError());
-          resolved.put(schema.getFullName(), createRecord);
-          final List<Schema.Field> currFields = schema.getFields();
-          List<Schema.Field> newFields = new ArrayList<Schema.Field>(currFields.size());
-          for (Schema.Field field : currFields) {
-            if (field.name().equals("hash")) {
-              System.err.println(field);
-            }
-            Schema.Field nf = new Schema.Field(field.name(), getResolvedSchema(field.schema(), resolved),
-                    field.doc(), field.defaultVal(), field.order());
-            Schemas.copyAliases(field, nf);
-            Schemas.copyProperties(field, nf);
-            newFields.add(nf);
-          }
-          createRecord.setFields(newFields);
-          Schemas.copyLogicalTypes(schema, createRecord);
-          Schemas.copyProperties(schema, createRecord);
-          return createRecord;
-        case MAP:
-          Schema createMap = Schema.createMap(getResolvedSchema(schema.getValueType(), resolved));
-          Schemas.copyProperties(schema, createMap);
-          return createMap;
-        case ARRAY:
-          Schema createArray = Schema.createArray(getResolvedSchema(schema.getElementType(), resolved));
-          Schemas.copyProperties(schema, createArray);
-          return createArray;
-        case UNION:
-          final List<Schema> uTypes = schema.getTypes();
-          List<Schema> newTypes = new ArrayList<Schema>(uTypes.size());
-          for (Schema s : uTypes) {
-            newTypes.add(getResolvedSchema(s, resolved));
-          }
-          Schema createUnion = Schema.createUnion(newTypes);
-          Schemas.copyProperties(schema, createUnion);
-          return createUnion;
-        case ENUM:
-        case FIXED:
-        case STRING:
-        case BYTES:
-        case INT:
-        case LONG:
-        case FLOAT:
-        case DOUBLE:
-        case BOOLEAN:
-        case NULL:
-          return schema;
-        default:
-          throw new RuntimeException("Unknown type: " + schema);
-      }
+
+    @Override
+    public Schema apply(final String f) {
+      return symbolTable.getType(f);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/avro/blob/874fff78/lang/java/compiler/src/main/java/org/apache/avro/compiler/schema/CloningVisitor.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/main/java/org/apache/avro/compiler/schema/CloningVisitor.java b/lang/java/compiler/src/main/java/org/apache/avro/compiler/schema/CloningVisitor.java
new file mode 100644
index 0000000..8c6423b
--- /dev/null
+++ b/lang/java/compiler/src/main/java/org/apache/avro/compiler/schema/CloningVisitor.java
@@ -0,0 +1,155 @@
+
+package org.apache.avro.compiler.schema;
+
+import java.util.ArrayList;
+import java.util.IdentityHashMap;
+import java.util.List;
+import org.apache.avro.Schema;
+import static org.apache.avro.Schema.Type.RECORD;
+
+/**
+ * this visitor will create a clone of the original Schema with docs and other nonesential fields stripped
+ * by default. what attributes are copied is customizable.
+ * @author zoly
+ */
+public final class CloningVisitor implements SchemaVisitor<Schema> {
+
+  private final IdentityHashMap<Schema, Schema> replace = new IdentityHashMap<Schema, Schema>();
+
+  private final Schema root;
+
+  private final PropertyCopier copyProperties;
+
+  private final boolean copyDocs;
+
+  public interface PropertyCopier {
+    void copy(Schema first, Schema second);
+    void copy(Schema.Field first, Schema.Field second);
+  }
+
+  /**
+   * copy only serialization necessary fields.
+   * @param root
+   */
+  public CloningVisitor(final Schema root) {
+    this(new PropertyCopier() {
+      @Override
+      public void copy(final Schema first, final Schema second) {
+        Schemas.copyLogicalTypes(first, second);
+        Schemas.copyAliases(first, second);
+      }
+
+      @Override
+      public void copy(final Schema.Field first, final Schema.Field second) {
+         Schemas.copyAliases(first, second);
+      }
+    }, false, root);
+  }
+
+  public CloningVisitor(final PropertyCopier copyProperties, final boolean copyDocs, final Schema root) {
+    this.copyProperties = copyProperties;
+    this.copyDocs = copyDocs;
+    this.root = root;
+  }
+
+  @Override
+  public SchemaVisitorAction visitTerminal(final Schema terminal) {
+    Schema.Type type = terminal.getType();
+    Schema newSchema;
+    switch (type) {
+      case RECORD: // recursion.
+      case ARRAY:
+      case MAP:
+      case UNION:
+        if (!replace.containsKey(terminal)) {
+          throw new IllegalStateException("Schema " + terminal + " must be already processed");
+        }
+        return SchemaVisitorAction.CONTINUE;
+      case BOOLEAN:
+      case BYTES:
+      case DOUBLE:
+      case FLOAT:
+      case INT:
+      case LONG:
+      case NULL:
+      case STRING:
+        newSchema = Schema.create(type);
+      break;
+      case ENUM:
+        newSchema = Schema.createEnum(terminal.getName(), copyDocs ? terminal.getDoc() : null,
+                terminal.getNamespace(), terminal.getEnumSymbols());
+        break;
+      case FIXED:
+        newSchema = Schema.createFixed(terminal.getName(), copyDocs ? terminal.getDoc() : null,
+                terminal.getNamespace(), terminal.getFixedSize());
+        break;
+      default:
+        throw new IllegalStateException("Unsupported schema " + terminal);
+    }
+    copyProperties.copy(terminal, newSchema);
+    replace.put(terminal, newSchema);
+    return SchemaVisitorAction.CONTINUE;
+  }
+
+  @Override
+  public SchemaVisitorAction visitNonTerminal(final Schema nt) {
+    Schema.Type type = nt.getType();
+    if  (type == RECORD) {
+        Schema newSchema = Schema.createRecord(nt.getName(), copyDocs ? nt.getDoc() : null,
+                nt.getNamespace(), nt.isError());
+        copyProperties.copy(nt, newSchema);
+        replace.put(nt, newSchema);
+    }
+    return SchemaVisitorAction.CONTINUE;
+  }
+
+  @Override
+  public SchemaVisitorAction afterVisitNonTerminal(final Schema nt) {
+     Schema.Type type = nt.getType();
+     Schema newSchema;
+     switch (type) {
+       case RECORD:
+         newSchema = replace.get(nt);
+         List<Schema.Field> fields = nt.getFields();
+         List<Schema.Field> newFields = new ArrayList<Schema.Field>(fields.size());
+         for (Schema.Field field : fields) {
+          Schema.Field newField = new Schema.Field(field.name(), replace.get(field.schema()),
+                  copyDocs ? field.doc() : null, field.defaultVal(), field.order());
+          copyProperties.copy(field, newField);
+          newFields.add(newField);
+         }
+         newSchema.setFields(newFields);
+         return SchemaVisitorAction.CONTINUE;
+       case UNION:
+          List<Schema> types = nt.getTypes();
+          List<Schema> newTypes = new ArrayList<Schema>(types.size());
+          for (Schema sch : types) {
+            newTypes.add(replace.get(sch));
+          }
+          newSchema = Schema.createUnion(newTypes);
+          break;
+       case ARRAY:
+         newSchema = Schema.createArray(replace.get(nt.getElementType()));
+         break;
+       case MAP:
+         newSchema = Schema.createMap(replace.get(nt.getValueType()));
+         break;
+       default:
+         throw new IllegalStateException("Illegal type " + type + ", schema " + nt);
+     }
+     copyProperties.copy(nt, newSchema);
+     replace.put(nt, newSchema);
+     return SchemaVisitorAction.CONTINUE;
+  }
+
+  @Override
+  public Schema get() {
+    return replace.get(root);
+  }
+
+  @Override
+  public String toString() {
+    return "CloningVisitor{" + "replace=" + replace + ", root=" + root + '}';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/avro/blob/874fff78/lang/java/compiler/src/test/idl/cycle.avdl
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/test/idl/cycle.avdl b/lang/java/compiler/src/test/idl/cycle.avdl
new file mode 100644
index 0000000..78b0fc1
--- /dev/null
+++ b/lang/java/compiler/src/test/idl/cycle.avdl
@@ -0,0 +1,24 @@
+@namespace("org.apache.avro.gen.test")
+protocol Cycle {
+
+    record Record1 {
+      string fString = "";
+      Record3 rec3;
+    }
+
+    record Record2 {
+      TestFixed fFixed;
+      int val;
+      union {null, Record1} fRec1;
+    }
+
+    record Record3 {
+      TestEnum fEnum;
+      Record2 rec2;
+    }
+
+    enum TestEnum { bla, blu }
+
+    fixed TestFixed(16);
+
+}

http://git-wip-us.apache.org/repos/asf/avro/blob/874fff78/lang/java/compiler/src/test/java/org/apache/avro/compiler/idl/SchemaResolverTest.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/test/java/org/apache/avro/compiler/idl/SchemaResolverTest.java b/lang/java/compiler/src/test/java/org/apache/avro/compiler/idl/SchemaResolverTest.java
new file mode 100644
index 0000000..61212f7
--- /dev/null
+++ b/lang/java/compiler/src/test/java/org/apache/avro/compiler/idl/SchemaResolverTest.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.compiler.idl;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import org.apache.avro.Protocol;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ *
+ * @author zoly
+ */
+public class SchemaResolverTest {
+
+
+  @Test
+  public void testResolving() throws ParseException, MalformedURLException, IOException {
+      File file = new File(".");
+      String currentWorkPath = file.getAbsolutePath();
+      String testIdl = currentWorkPath + File.separator + "src" + File.separator + "test"
+        + File.separator + "idl" + File.separator + "cycle.avdl";
+      Idl compiler = new Idl(new File(testIdl));
+      Protocol protocol = compiler.CompilationUnit();
+      System.out.println(protocol);
+      Assert.assertEquals(5, protocol.getTypes().size());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/avro/blob/874fff78/lang/java/compiler/src/test/java/org/apache/avro/compiler/schema/SchemasTest.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/test/java/org/apache/avro/compiler/schema/SchemasTest.java b/lang/java/compiler/src/test/java/org/apache/avro/compiler/schema/SchemasTest.java
new file mode 100644
index 0000000..545901e
--- /dev/null
+++ b/lang/java/compiler/src/test/java/org/apache/avro/compiler/schema/SchemasTest.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.compiler.schema;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaCompatibility;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SchemasTest {
+
+  private static final String SCHEMA = "{\"type\":\"record\",\"name\":\"SampleNode\",\"doc\":\"caca\","
+      + "\"namespace\":\"org.spf4j.ssdump2.avro\",\n" +
+      " \"fields\":[\n" +
+      "    {\"name\":\"count\",\"type\":\"int\",\"default\":0,\"doc\":\"caca\"},\n" +
+      "    {\"name\":\"subNodes\",\"type\":\n" +
+      "       {\"type\":\"array\",\"items\":{\n" +
+      "           \"type\":\"record\",\"name\":\"SamplePair\",\n" +
+      "           \"fields\":[\n" +
+      "              {\"name\":\"method\",\"type\":\n" +
+      "                  {\"type\":\"record\",\"name\":\"Method\",\n" +
+      "                  \"fields\":[\n" +
+      "                     {\"name\":\"declaringClass\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},\n" +
+      "                     {\"name\":\"methodName\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}\n" +
+      "                  ]}},\n" +
+      "              {\"name\":\"node\",\"type\":\"SampleNode\"}]}}}]}";
+
+  private static class PrintingVisitor implements SchemaVisitor {
+
+
+    @Override
+    public SchemaVisitorAction visitTerminal(Schema terminal) {
+      System.out.println("Terminal: " + terminal.getFullName());
+      return SchemaVisitorAction.CONTINUE;
+    }
+
+    @Override
+    public SchemaVisitorAction visitNonTerminal(Schema terminal) {
+      System.out.println("NONTerminal start: " + terminal.getFullName());
+      return SchemaVisitorAction.CONTINUE;
+    }
+
+    @Override
+    public SchemaVisitorAction afterVisitNonTerminal(Schema terminal) {
+      System.out.println("NONTerminal end: " + terminal.getFullName());
+      return SchemaVisitorAction.CONTINUE;
+    }
+
+    @Override
+    public Object get() {
+      return null;
+    }
+  }
+
+
+
+  @Test
+  public void textCloning() {
+    Schema recSchema = new Schema.Parser().parse(SCHEMA);
+    Schemas.visit(recSchema, new PrintingVisitor());
+
+
+    Schema trimmed = Schemas.visit(recSchema, new CloningVisitor(recSchema));
+    Assert.assertNull(trimmed.getDoc());
+    Assert.assertNotNull(recSchema.getDoc());
+
+    SchemaCompatibility.SchemaCompatibilityType compat =
+            SchemaCompatibility.checkReaderWriterCompatibility(trimmed, recSchema).getType();
+    Assert.assertEquals(SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE, compat);
+    compat = SchemaCompatibility.checkReaderWriterCompatibility(recSchema, trimmed).getType();
+    Assert.assertEquals(SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE, compat);
+  }
+
+}