You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by bl...@apache.org on 2015/04/28 01:12:25 UTC

[28/51] [partial] parquet-mr git commit: PARQUET-23: Rename to org.apache.parquet.

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/io/api/RecordMaterializer.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/io/api/RecordMaterializer.java b/parquet-column/src/main/java/parquet/io/api/RecordMaterializer.java
deleted file mode 100644
index 69e96ef..0000000
--- a/parquet-column/src/main/java/parquet/io/api/RecordMaterializer.java
+++ /dev/null
@@ -1,48 +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 parquet.io.api;
-
-/**
- * Top-level class which should be implemented in order to materialize objects from
- * a stream of Parquet data.
- * 
- * Each record will be wrapped by {@link GroupConverter#start()} and {@link GroupConverter#end()},
- * between which the appropriate fields will be materialized.
- *
- * @author Julien Le Dem
- *
- * @param <T> the materialized object class
- */
-abstract public class RecordMaterializer<T> {
-
-  /**
-   * @return the result of the conversion
-   */
-  abstract public T getCurrentRecord();
-
-  /**
-   * Called if {@link #getCurrentRecord()} isn't going to be called.
-   */
-  public void skipCurrentRecord() { }
-
-  /**
-   * @return the root converter for this tree
-   */
-  abstract public GroupConverter getRootConverter();
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/ConversionPatterns.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/ConversionPatterns.java b/parquet-column/src/main/java/parquet/schema/ConversionPatterns.java
deleted file mode 100644
index cfb8448..0000000
--- a/parquet-column/src/main/java/parquet/schema/ConversionPatterns.java
+++ /dev/null
@@ -1,104 +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 parquet.schema;
-
-import parquet.schema.PrimitiveType.PrimitiveTypeName;
-import parquet.schema.Type.Repetition;
-
-import static parquet.schema.OriginalType.*;
-
-/**
- * Utility functions to convert from Java-like map and list types
- * to equivalent Parquet types.
- */
-public abstract class ConversionPatterns {
-  /**
-   * to preserve the difference between empty list and null when optional
-   *
-   * @param repetition
-   * @param alias        name of the field
-   * @param originalType
-   * @param nested       the nested repeated field
-   * @return a group type
-   */
-  private static GroupType listWrapper(Repetition repetition, String alias, OriginalType originalType, Type nested) {
-    if (!nested.isRepetition(Repetition.REPEATED)) {
-      throw new IllegalArgumentException("Nested type should be repeated: " + nested);
-    }
-    return new GroupType(repetition, alias, originalType, nested);
-  }
-
-  public static GroupType mapType(Repetition repetition, String alias, Type keyType, Type valueType) {
-    return mapType(repetition, alias, "map", keyType, valueType);
-  }
-
-  public static GroupType stringKeyMapType(Repetition repetition, String alias, String mapAlias, Type valueType) {
-    return mapType(repetition, alias, mapAlias, new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "key", OriginalType.UTF8), valueType);
-  }
-
-  public static GroupType stringKeyMapType(Repetition repetition, String alias, Type valueType) {
-    return stringKeyMapType(repetition, alias, "map", valueType);
-  }
-
-  public static GroupType mapType(Repetition repetition, String alias, String mapAlias, Type keyType, Type valueType) {
-    //support projection only on key of a map
-    if (valueType == null) {
-      return listWrapper(
-              repetition,
-              alias,
-              MAP,
-              new GroupType(
-                      Repetition.REPEATED,
-                      mapAlias,
-                      MAP_KEY_VALUE,
-                      keyType)
-      );
-    } else {
-      if (!valueType.getName().equals("value")) {
-        throw new RuntimeException(valueType.getName() + " should be value");
-      }
-      return listWrapper(
-              repetition,
-              alias,
-              MAP,
-              new GroupType(
-                      Repetition.REPEATED,
-                      mapAlias,
-                      MAP_KEY_VALUE,
-                      keyType,
-                      valueType)
-      );
-    }
-  }
-
-  /**
-   * @param repetition
-   * @param alias      name of the field
-   * @param nestedType
-   * @return
-   */
-  public static GroupType listType(Repetition repetition, String alias, Type nestedType) {
-    return listWrapper(
-            repetition,
-            alias,
-            LIST,
-            nestedType
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/DecimalMetadata.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/DecimalMetadata.java b/parquet-column/src/main/java/parquet/schema/DecimalMetadata.java
deleted file mode 100644
index b9eca4d..0000000
--- a/parquet-column/src/main/java/parquet/schema/DecimalMetadata.java
+++ /dev/null
@@ -1,57 +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 parquet.schema;
-
-public class DecimalMetadata {
-  private final int precision;
-  private final int scale;
-
-  public DecimalMetadata(int precision, int scale) {
-    this.precision = precision;
-    this.scale = scale;
-  }
-
-  public int getPrecision() {
-    return precision;
-  }
-
-  public int getScale() {
-    return scale;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    DecimalMetadata that = (DecimalMetadata) o;
-
-    if (precision != that.precision) return false;
-    if (scale != that.scale) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = precision;
-    result = 31 * result + scale;
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/GroupType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/GroupType.java b/parquet-column/src/main/java/parquet/schema/GroupType.java
deleted file mode 100644
index 074e572..0000000
--- a/parquet-column/src/main/java/parquet/schema/GroupType.java
+++ /dev/null
@@ -1,391 +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 parquet.schema;
-
-import static java.util.Arrays.asList;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import parquet.io.InvalidRecordException;
-
-/**
- * Represents a group type: a list of fields
- *
- * @author Julien Le Dem
- *
- */
-public class GroupType extends Type {
-
-  private final List<Type> fields;
-  private final Map<String, Integer> indexByName;
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param name the name of the field
-   * @param fields the contained fields
-   */
-  public GroupType(Repetition repetition, String name, List<Type> fields) {
-    this(repetition, name, null, fields, null);
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param name the name of the field
-   * @param fields the contained fields
-   */
-  public GroupType(Repetition repetition, String name, Type... fields) {
-    this(repetition, name, Arrays.asList(fields));
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param name the name of the field
-   * @param originalType (optional) the original type to help with cross schema conversion (LIST, MAP, ...)
-   * @param fields the contained fields
-   */
-  @Deprecated
-  public GroupType(Repetition repetition, String name, OriginalType originalType, Type... fields) {
-    this(repetition, name, originalType, Arrays.asList(fields));
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param name the name of the field
-   * @param originalType (optional) the original type to help with cross schema conversion (LIST, MAP, ...)
-   * @param fields the contained fields
-   */
-  @Deprecated
-  public GroupType(Repetition repetition, String name, OriginalType originalType, List<Type> fields) {
-    this(repetition, name, originalType, fields, null);
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param name the name of the field
-   * @param originalType (optional) the original type to help with cross schema conversion (LIST, MAP, ...)
-   * @param fields the contained fields
-   * @param id the id of the field
-   */
-  GroupType(Repetition repetition, String name, OriginalType originalType, List<Type> fields, ID id) {
-    super(name, repetition, originalType, id);
-    this.fields = fields;
-    this.indexByName = new HashMap<String, Integer>();
-    for (int i = 0; i < fields.size(); i++) {
-      indexByName.put(fields.get(i).getName(), i);
-    }
-  }
-
-  /**
-   * @param id the field id
-   * @return a new GroupType with the same fields and a new id
-   */
-  @Override
-  public GroupType withId(int id) {
-    return new GroupType(getRepetition(), getName(), getOriginalType(), fields, new ID(id));
-  }
-
-  /**
-   * @param newFields
-   * @return a group with the same attributes and new fields.
-   */
-  public GroupType withNewFields(List<Type> newFields) {
-    return new GroupType(getRepetition(), getName(), getOriginalType(), newFields, getId());
-  }
-
-  /**
-   * @param newFields
-   * @return a group with the same attributes and new fields.
-   */
-  public GroupType withNewFields(Type... newFields) {
-    return withNewFields(asList(newFields));
-  }
-
-  /**
-   * returns the name of the corresponding field
-   * @param index the index of the desired field in this type
-   * @return the name of the field at this index
-   */
-  public String getFieldName(int index) {
-    return fields.get(index).getName();
-  }
-
-  /**
-   * @param name the requested name
-   * @return whether this type contains a field with that name
-   */
-  public boolean containsField(String name) {
-    return indexByName.containsKey(name);
-  }
-
-  /**
-   *
-   * @param name
-   * @return the index of the field with that name
-   */
-  public int getFieldIndex(String name) {
-    if (!indexByName.containsKey(name)) {
-      throw new InvalidRecordException(name + " not found in " + this);
-    }
-    return indexByName.get(name);
-  }
-
-  /**
-   * @return the fields contained in this type
-   */
-  public List<Type> getFields() {
-    return fields;
-  }
-
-  /**
-   * @return the number of fields in this type
-   */
-  public int getFieldCount() {
-    return fields.size();
-  }
-
-  /**
-   * @return false
-   */
-  @Override
-  public boolean isPrimitive() {
-    return false;
-  }
-
-  /**
-   * @param fieldName
-   * @return the type of this field by name
-   */
-  public Type getType(String fieldName) {
-    return getType(getFieldIndex(fieldName));
-  }
-
-  /**
-   * @param index
-   * @return the type of this field by index
-   */
-  public Type getType(int index) {
-    return fields.get(index);
-  }
-
-  /**
-   * appends a display string for of the members of this group to sb
-   * @param sb where to append
-   * @param indent the indentation level
-   */
-  void membersDisplayString(StringBuilder sb, String indent) {
-    for (Type field : fields) {
-      field.writeToStringBuilder(sb, indent);
-      if (field.isPrimitive()) {
-        sb.append(";");
-      }
-      sb.append("\n");
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void writeToStringBuilder(StringBuilder sb, String indent) {
-    sb.append(indent)
-        .append(getRepetition().name().toLowerCase())
-        .append(" group ")
-        .append(getName())
-        .append(getOriginalType() == null ? "" : " (" + getOriginalType() +")")
-        .append(getId() == null ? "" : " = " + getId())
-        .append(" {\n");
-    membersDisplayString(sb, indent + "  ");
-    sb.append(indent)
-        .append("}");
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void accept(TypeVisitor visitor) {
-    visitor.visit(this);
-  }
-
-  @Override @Deprecated
-  protected int typeHashCode() {
-    return hashCode();
-  }
-
-  @Override @Deprecated
-  protected boolean typeEquals(Type other) {
-    return equals(other);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public int hashCode() {
-    return super.hashCode() * 31 + getFields().hashCode();
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  protected boolean equals(Type otherType) {
-    return
-        !otherType.isPrimitive()
-        && super.equals(otherType)
-        && getFields().equals(otherType.asGroupType().getFields());
-  }
-
-  @Override
-  protected int getMaxRepetitionLevel(String[] path, int depth) {
-    int myVal = isRepetition(Repetition.REPEATED) ? 1 : 0;
-    if (depth == path.length) {
-      return myVal;
-    }
-    return myVal + getType(path[depth]).getMaxRepetitionLevel(path, depth + 1);
-  }
-
-  @Override
-  protected int getMaxDefinitionLevel(String[] path, int depth) {
-    int myVal = !isRepetition(Repetition.REQUIRED) ? 1 : 0;
-    if (depth == path.length) {
-      return myVal;
-    }
-    return myVal + getType(path[depth]).getMaxDefinitionLevel(path, depth + 1);
-  }
-
-  @Override
-  protected Type getType(String[] path, int depth) {
-    if (depth == path.length) {
-      return this;
-    }
-    return getType(path[depth]).getType(path, depth + 1);
-  }
-
-  @Override
-  protected boolean containsPath(String[] path, int depth) {
-    if (depth == path.length) {
-      return false;
-    }
-    return containsField(path[depth]) && getType(path[depth]).containsPath(path, depth + 1);
-  }
-
-  @Override
-  protected List<String[]> getPaths(int depth) {
-    List<String[]> result = new ArrayList<String[]>();
-    for (Type field : fields) {
-      List<String[]> paths = field.getPaths(depth + 1);
-      for (String[] path : paths) {
-        path[depth] = field.getName();
-        result.add(path);
-      }
-    }
-    return result;
-  }
-
-  @Override
-  void checkContains(Type subType) {
-    super.checkContains(subType);
-    checkGroupContains(subType);
-  }
-
-  void checkGroupContains(Type subType) {
-    if (subType.isPrimitive()) {
-      throw new InvalidRecordException(subType + " found: expected " + this);
-    }
-    List<Type> fields = subType.asGroupType().getFields();
-    for (Type otherType : fields) {
-      Type thisType = this.getType(otherType.getName());
-      thisType.checkContains(otherType);
-    }
-  }
-
-  @Override
-  <T> T convert(List<GroupType> path, TypeConverter<T> converter) {
-    List<GroupType> childrenPath = new ArrayList<GroupType>(path);
-    childrenPath.add(this);
-    final List<T> children = convertChildren(childrenPath, converter);
-    return converter.convertGroupType(path, this, children);
-  }
-
-  protected <T> List<T> convertChildren(List<GroupType> path, TypeConverter<T> converter) {
-    List<T> children = new ArrayList<T>(fields.size());
-    for (Type field : fields) {
-      children.add(field.convert(path, converter));
-    }
-    return children;
-  }
-
-  @Override
-  protected Type union(Type toMerge) {
-    return union(toMerge, true);
-  }
-
-  @Override
-  protected Type union(Type toMerge, boolean strict) {
-    if (toMerge.isPrimitive()) {
-      throw new IncompatibleSchemaModificationException("can not merge primitive type " + toMerge + " into group type " + this);
-    }
-    return new GroupType(toMerge.getRepetition(), getName(), mergeFields(toMerge.asGroupType()));
-  }
-
-  /**
-   * produces the list of fields resulting from merging toMerge into the fields of this
-   * @param toMerge the group containing the fields to merge
-   * @return the merged list
-   */
-  List<Type> mergeFields(GroupType toMerge) {
-    return mergeFields(toMerge, true);
-  }
-
-  /**
-   * produces the list of fields resulting from merging toMerge into the fields of this
-   * @param toMerge the group containing the fields to merge
-   * @param strict should schema primitive types match
-   * @return the merged list
-   */
-  List<Type> mergeFields(GroupType toMerge, boolean strict) {
-    List<Type> newFields = new ArrayList<Type>();
-    // merge existing fields
-    for (Type type : this.getFields()) {
-      Type merged;
-      if (toMerge.containsField(type.getName())) {
-        Type fieldToMerge = toMerge.getType(type.getName());
-        if (fieldToMerge.getRepetition().isMoreRestrictiveThan(type.getRepetition())) {
-          throw new IncompatibleSchemaModificationException("repetition constraint is more restrictive: can not merge type " + fieldToMerge + " into " + type);
-        }
-        merged = type.union(fieldToMerge, strict);
-      } else {
-        merged = type;
-      }
-      newFields.add(merged);
-    }
-    // add new fields
-    for (Type type : toMerge.getFields()) {
-      if (!this.containsField(type.getName())) {
-        newFields.add(type);
-      }
-    }
-    return newFields;
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/IncompatibleSchemaModificationException.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/IncompatibleSchemaModificationException.java b/parquet-column/src/main/java/parquet/schema/IncompatibleSchemaModificationException.java
deleted file mode 100644
index 4933294..0000000
--- a/parquet-column/src/main/java/parquet/schema/IncompatibleSchemaModificationException.java
+++ /dev/null
@@ -1,49 +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 parquet.schema;
-
-import parquet.ParquetRuntimeException;
-
-/**
- * thrown when we are trying to read together files with incompatible schemas.
- *
- * @author Julien Le Dem
- *
- */
-public class IncompatibleSchemaModificationException extends ParquetRuntimeException {
-  private static final long serialVersionUID = 1L;
-
-  public IncompatibleSchemaModificationException() {
-    super();
-  }
-
-  public IncompatibleSchemaModificationException(String message,
-      Throwable cause) {
-    super(message, cause);
-  }
-
-  public IncompatibleSchemaModificationException(String message) {
-    super(message);
-  }
-
-  public IncompatibleSchemaModificationException(Throwable cause) {
-    super(cause);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/MessageType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/MessageType.java b/parquet-column/src/main/java/parquet/schema/MessageType.java
deleted file mode 100644
index 7c52b65..0000000
--- a/parquet-column/src/main/java/parquet/schema/MessageType.java
+++ /dev/null
@@ -1,148 +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 parquet.schema;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import parquet.column.ColumnDescriptor;
-import parquet.io.InvalidRecordException;
-
-/**
- * The root of a schema
- *
- * @author Julien Le Dem
- *
- */
-public final class MessageType extends GroupType {
-
-  /**
-   *
-   * @param name the name of the type
-   * @param fields the fields contained by this message
-   */
-  public MessageType(String name, Type... fields) {
-    super(Repetition.REPEATED, name, fields);
-  }
-
- /**
-  *
-  * @param name the name of the type
-  * @param fields the fields contained by this message
-  */
- public MessageType(String name, List<Type> fields) {
-   super(Repetition.REPEATED, name, fields);
- }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void accept(TypeVisitor visitor) {
-    visitor.visit(this);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void writeToStringBuilder(StringBuilder sb, String indent) {
-    sb.append("message ")
-        .append(getName())
-        .append(getOriginalType() == null ? "" : " (" + getOriginalType() +")")
-        .append(" {\n");
-    membersDisplayString(sb, "  ");
-    sb.append("}\n");
-  }
-
-  /**
-   * @return the max repetition level that might be needed to encode the
-   * type at 'path'.
-   */
-  public int getMaxRepetitionLevel(String ... path) {
-    return getMaxRepetitionLevel(path, 0) - 1;
-  }
-
-  /**
-   * @return the max repetition level that might be needed to encode the
-   * type at 'path'.
-   */
-  public int getMaxDefinitionLevel(String ... path) {
-    return getMaxDefinitionLevel(path, 0) - 1;
-  }
-
-  public Type getType(String ... path) {
-    return getType(path, 0);
-  }
-
-  public ColumnDescriptor getColumnDescription(String[] path) {
-    int maxRep = getMaxRepetitionLevel(path);
-    int maxDef = getMaxDefinitionLevel(path);
-    PrimitiveType type = getType(path).asPrimitiveType();
-    return new ColumnDescriptor(path, type.getPrimitiveTypeName(),
-                                type.getTypeLength(), maxRep, maxDef);
-  }
-
-  public List<String[]> getPaths() {
-    return this.getPaths(0);
-  }
-
-  public List<ColumnDescriptor> getColumns() {
-    List<String[]> paths = this.getPaths(0);
-    List<ColumnDescriptor> columns = new ArrayList<ColumnDescriptor>(paths.size());
-    for (String[] path : paths) {
-      // TODO: optimize this
-      PrimitiveType primitiveType = getType(path).asPrimitiveType();
-      columns.add(new ColumnDescriptor(
-                      path,
-                      primitiveType.getPrimitiveTypeName(),
-                      primitiveType.getTypeLength(),
-                      getMaxRepetitionLevel(path),
-                      getMaxDefinitionLevel(path)));
-    }
-    return columns;
-  }
-
-  @Override
-  public void checkContains(Type subType) {
-    if (!(subType instanceof MessageType)) {
-      throw new InvalidRecordException(subType + " found: expected " + this);
-    }
-    checkGroupContains(subType);
-  }
-
-  public <T> T convertWith(TypeConverter<T> converter) {
-    final ArrayList<GroupType> path = new ArrayList<GroupType>();
-    path.add(this);
-    return converter.convertMessageType(this, convertChildren(path, converter));
-  }
-
-  public boolean containsPath(String[] path) {
-    return containsPath(path, 0);
-  }
-
-  public MessageType union(MessageType toMerge) {
-    return union(toMerge, true);
-  }
-
-  public MessageType union(MessageType toMerge, boolean strict) {
-    return new MessageType(this.getName(), mergeFields(toMerge, strict));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/MessageTypeParser.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/MessageTypeParser.java b/parquet-column/src/main/java/parquet/schema/MessageTypeParser.java
deleted file mode 100644
index 7afc5fb..0000000
--- a/parquet-column/src/main/java/parquet/schema/MessageTypeParser.java
+++ /dev/null
@@ -1,216 +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 parquet.schema;
-
-import java.util.Arrays;
-import java.util.StringTokenizer;
-
-import parquet.Log;
-import parquet.schema.PrimitiveType.PrimitiveTypeName;
-import parquet.schema.Type.Repetition;
-import parquet.schema.Types.GroupBuilder;
-import parquet.schema.Types.PrimitiveBuilder;
-
-/**
- * Parses a schema from a textual format similar to that described in the Dremel paper.
- *
- * @author Julien Le Dem
- */
-public class MessageTypeParser {
-  private static final Log LOG = Log.getLog(MessageTypeParser.class);
-
-  private static class Tokenizer {
-
-    private StringTokenizer st;
-
-    private int line = 0;
-    private StringBuffer currentLine = new StringBuffer();
-
-    public Tokenizer(String schemaString, String string) {
-      st = new StringTokenizer(schemaString, " ,;{}()\n\t=", true);
-    }
-
-    public String nextToken() {
-      while (st.hasMoreTokens()) {
-        String t = st.nextToken();
-        if (t.equals("\n")) {
-          ++ line;
-          currentLine.setLength(0);
-        } else {
-          currentLine.append(t);
-        }
-        if (!isWhitespace(t)) {
-          return t;
-        }
-      }
-      throw new IllegalArgumentException("unexpected end of schema");
-    }
-
-    private boolean isWhitespace(String t) {
-      return t.equals(" ") || t.equals("\t") || t.equals("\n");
-    }
-
-    public String getLocationString() {
-      return "line " + line + ": " + currentLine.toString();
-    }
-  }
-
-  private MessageTypeParser() {}
-
-  /**
-   *
-   * @param input the text representation of the schema to parse
-   * @return the corresponding object representation
-   */
-  public static MessageType parseMessageType(String input) {
-    return parse(input);
-  }
-
-  private static MessageType parse(String schemaString) {
-    Tokenizer st = new Tokenizer(schemaString, " ;{}()\n\t");
-    Types.MessageTypeBuilder builder = Types.buildMessage();
-
-    String t = st.nextToken();
-    check(t, "message", "start with 'message'", st);
-    String name = st.nextToken();
-    addGroupTypeFields(st.nextToken(), st, builder);
-    return builder.named(name);
-  }
-
-  private static void addGroupTypeFields(String t, Tokenizer st, Types.GroupBuilder builder) {
-    check(t, "{", "start of message", st);
-    while (!(t = st.nextToken()).equals("}")) {
-      addType(t, st, builder);
-    }
-  }
-
-  private static void addType(String t, Tokenizer st, Types.GroupBuilder builder) {
-    Repetition repetition = asRepetition(t, st);
-
-    // Read type.
-    String type = st.nextToken();
-    if ("group".equalsIgnoreCase(type)) {
-      addGroupType(t, st, repetition, builder);
-    } else {
-      addPrimitiveType(t, st, asPrimitive(type, st), repetition, builder);
-    }
-  }
-
-  private static void addGroupType(String t, Tokenizer st, Repetition r, GroupBuilder<?> builder) {
-    GroupBuilder<?> childBuilder = builder.group(r);
-    String name = st.nextToken();
-
-    // Read annotation, if any.
-    t = st.nextToken();
-    OriginalType originalType = null;
-    if (t.equalsIgnoreCase("(")) {
-      originalType = OriginalType.valueOf(st.nextToken());
-      childBuilder.as(originalType);
-      check(st.nextToken(), ")", "original type ended by )", st);
-      t = st.nextToken();
-    }
-    if (t.equals("=")) {
-      childBuilder.id(Integer.parseInt(st.nextToken()));
-      t = st.nextToken();
-    }
-    try {
-      addGroupTypeFields(t, st, childBuilder);
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException("problem reading type: type = group, name = " + name + ", original type = " + originalType, e);
-    }
-
-    childBuilder.named(name);
-  }
-
-  private static void addPrimitiveType(String t, Tokenizer st, PrimitiveTypeName type, Repetition r, Types.GroupBuilder<?> builder) {
-    PrimitiveBuilder<?> childBuilder = builder.primitive(type, r);
-
-    if (type == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
-      t = st.nextToken();
-      // Read type length if the type is fixed_len_byte_array.
-      if (!t.equalsIgnoreCase("(")) {
-        throw new IllegalArgumentException("expecting (length) for field of type fixed_len_byte_array");
-      }
-      childBuilder.length(Integer.parseInt(st.nextToken()));
-      check(st.nextToken(), ")", "type length ended by )", st);
-    }
-
-    String name = st.nextToken();
-
-    // Read annotation, if any.
-    t = st.nextToken();
-    OriginalType originalType = null;
-    if (t.equalsIgnoreCase("(")) {
-      originalType = OriginalType.valueOf(st.nextToken());
-      childBuilder.as(originalType);
-      if (OriginalType.DECIMAL == originalType) {
-        t = st.nextToken();
-        // parse precision and scale
-        if (t.equalsIgnoreCase("(")) {
-          childBuilder.precision(Integer.parseInt(st.nextToken()));
-          t = st.nextToken();
-          if (t.equalsIgnoreCase(",")) {
-            childBuilder.scale(Integer.parseInt(st.nextToken()));
-            t = st.nextToken();
-          }
-          check(t, ")", "decimal type ended by )", st);
-          t = st.nextToken();
-        }
-      } else {
-        t = st.nextToken();
-      }
-      check(t, ")", "original type ended by )", st);
-      t = st.nextToken();
-    }
-    if (t.equals("=")) {
-      childBuilder.id(Integer.parseInt(st.nextToken()));
-      t = st.nextToken();
-    }
-    check(t, ";", "field ended by ';'", st);
-
-    try {
-      childBuilder.named(name);
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException("problem reading type: type = " + type + ", name = " + name + ", original type = " + originalType, e);
-    }
-  }
-
-  private static PrimitiveTypeName asPrimitive(String t, Tokenizer st) {
-    try {
-      return PrimitiveTypeName.valueOf(t.toUpperCase());
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException("expected one of " + Arrays.toString(PrimitiveTypeName.values())  +" got " + t + " at " + st.getLocationString(), e);
-    }
-  }
-
-  private static Repetition asRepetition(String t, Tokenizer st) {
-    try {
-      return Repetition.valueOf(t.toUpperCase());
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException("expected one of " + Arrays.toString(Repetition.values())  +" got " + t + " at " + st.getLocationString(), e);
-    }
-  }
-
-  private static void check(String t, String expected, String message, Tokenizer tokenizer) {
-    if (!t.equalsIgnoreCase(expected)) {
-      throw new IllegalArgumentException(message+ ": expected '" + expected + "' but got '" + t + "' at " + tokenizer.getLocationString());
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/OriginalType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/OriginalType.java b/parquet-column/src/main/java/parquet/schema/OriginalType.java
deleted file mode 100644
index aafa17b..0000000
--- a/parquet-column/src/main/java/parquet/schema/OriginalType.java
+++ /dev/null
@@ -1,42 +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 parquet.schema;
-
-public enum OriginalType {
-  MAP,
-  LIST,
-  UTF8,
-  MAP_KEY_VALUE,
-  ENUM,
-  DECIMAL,
-  DATE,
-  TIME_MILLIS,
-  TIMESTAMP_MILLIS,
-  UINT_8,
-  UINT_16,
-  UINT_32,
-  UINT_64,
-  INT_8,
-  INT_16,
-  INT_32,
-  INT_64,
-  JSON,
-  BSON,
-  INTERVAL;
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/PrimitiveType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/PrimitiveType.java b/parquet-column/src/main/java/parquet/schema/PrimitiveType.java
deleted file mode 100644
index 4d47ce5..0000000
--- a/parquet-column/src/main/java/parquet/schema/PrimitiveType.java
+++ /dev/null
@@ -1,530 +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 parquet.schema;
-
-import java.util.Arrays;
-import java.util.List;
-
-import parquet.column.ColumnReader;
-import parquet.io.InvalidRecordException;
-import parquet.io.api.Binary;
-import parquet.io.api.PrimitiveConverter;
-import parquet.io.api.RecordConsumer;
-
-
-/**
- *
- * Representation of a Primitive type
- *
- * @author Julien Le Dem
- *
- */
-public final class PrimitiveType extends Type {
-
-  public static interface PrimitiveTypeNameConverter<T, E extends Exception> {
-
-    T convertFLOAT(PrimitiveTypeName primitiveTypeName) throws E;
-
-    T convertDOUBLE(PrimitiveTypeName primitiveTypeName) throws E;
-
-    T convertINT32(PrimitiveTypeName primitiveTypeName) throws E;
-
-    T convertINT64(PrimitiveTypeName primitiveTypeName) throws E;
-
-    T convertINT96(PrimitiveTypeName primitiveTypeName) throws E;
-
-    T convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) throws E;
-
-    T convertBOOLEAN(PrimitiveTypeName primitiveTypeName) throws E;
-
-    T convertBINARY(PrimitiveTypeName primitiveTypeName) throws E;
-
-  }
-
-  /**
-   * Supported Primitive types
-   *
-   * @author Julien Le Dem
-   */
-  public static enum PrimitiveTypeName {
-    INT64("getLong", Long.TYPE) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return String.valueOf(columnReader.getLong());
-      }
-
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addLong(columnReader.getLong());
-      }
-
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addLong(columnReader.getLong());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertINT64(this);
-      }
-    },
-    INT32("getInteger", Integer.TYPE) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return String.valueOf(columnReader.getInteger());
-      }
-
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addInteger(columnReader.getInteger());
-      }
-
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addInt(columnReader.getInteger());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertINT32(this);
-      }
-    },
-    BOOLEAN("getBoolean", Boolean.TYPE) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return String.valueOf(columnReader.getBoolean());
-      }
-
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addBoolean(columnReader.getBoolean());
-      }
-
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addBoolean(columnReader.getBoolean());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertBOOLEAN(this);
-      }
-    },
-    BINARY("getBinary", Binary.class) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return String.valueOf(columnReader.getBinary());
-      }
-
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addBinary(columnReader.getBinary());
-      }
-
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addBinary(columnReader.getBinary());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertBINARY(this);
-      }
-    },
-    FLOAT("getFloat", Float.TYPE) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return String.valueOf(columnReader.getFloat());
-      }
-
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addFloat(columnReader.getFloat());
-      }
-
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addFloat(columnReader.getFloat());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertFLOAT(this);
-      }
-    },
-    DOUBLE("getDouble", Double.TYPE) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return String.valueOf(columnReader.getDouble());
-      }
-
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addDouble(columnReader.getDouble());
-      }
-
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addDouble(columnReader.getDouble());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertDOUBLE(this);
-      }
-    },
-    INT96("getBinary", Binary.class) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return Arrays.toString(columnReader.getBinary().getBytes());
-      }
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addBinary(columnReader.getBinary());
-      }
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addBinary(columnReader.getBinary());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertINT96(this);
-      }
-    },
-    FIXED_LEN_BYTE_ARRAY("getBinary", Binary.class) {
-      @Override
-      public String toString(ColumnReader columnReader) {
-        return String.valueOf(columnReader.getBinary());
-      }
-
-      @Override
-      public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-          ColumnReader columnReader) {
-        recordConsumer.addBinary(columnReader.getBinary());
-      }
-
-      @Override
-      public void addValueToPrimitiveConverter(
-          PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
-        primitiveConverter.addBinary(columnReader.getBinary());
-      }
-
-      @Override
-      public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E {
-        return converter.convertFIXED_LEN_BYTE_ARRAY(this);
-      }
-    };
-
-    public final String getMethod;
-    public final Class<?> javaType;
-
-    private PrimitiveTypeName(String getMethod, Class<?> javaType) {
-      this.getMethod = getMethod;
-      this.javaType = javaType;
-    }
-
-    /**
-     * reads the value from the columnReader with the appropriate accessor and returns a String representation
-     * @param columnReader
-     * @return a string
-     */
-    abstract public String toString(ColumnReader columnReader);
-
-    /**
-     * reads the value from the columnReader with the appropriate accessor and writes it to the recordConsumer
-     * @param recordConsumer where to write
-     * @param columnReader where to read from
-     */
-    abstract public void addValueToRecordConsumer(RecordConsumer recordConsumer,
-        ColumnReader columnReader);
-
-    abstract public void addValueToPrimitiveConverter(
-        PrimitiveConverter primitiveConverter, ColumnReader columnReader);
-
-    abstract public <T, E extends Exception> T convert(PrimitiveTypeNameConverter<T, E> converter) throws E;
-
-  }
-
-  private final PrimitiveTypeName primitive;
-  private final int length;
-  private final DecimalMetadata decimalMeta;
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param primitive STRING, INT64, ...
-   * @param name the name of the type
-   */
-  public PrimitiveType(Repetition repetition, PrimitiveTypeName primitive,
-                       String name) {
-    this(repetition, primitive, 0, name, null, null, null);
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param primitive STRING, INT64, ...
-   * @param length the length if the type is FIXED_LEN_BYTE_ARRAY, 0 otherwise (XXX)
-   * @param name the name of the type
-   */
-  public PrimitiveType(Repetition repetition, PrimitiveTypeName primitive, int length, String name) {
-    this(repetition, primitive, length, name, null, null, null);
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param primitive STRING, INT64, ...
-   * @param name the name of the type
-   * @param originalType (optional) the original type to help with cross schema convertion (LIST, MAP, ...)
-   */
-  public PrimitiveType(Repetition repetition, PrimitiveTypeName primitive,
-                       String name, OriginalType originalType) {
-    this(repetition, primitive, 0, name, originalType, null, null);
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param primitive STRING, INT64, ...
-   * @param name the name of the type
-   * @param length the length if the type is FIXED_LEN_BYTE_ARRAY, 0 otherwise (XXX)
-   * @param originalType (optional) the original type to help with cross schema conversion (LIST, MAP, ...)
-   */
-  @Deprecated
-  public PrimitiveType(Repetition repetition, PrimitiveTypeName primitive,
-                       int length, String name, OriginalType originalType) {
-    this(repetition, primitive, length, name, originalType, null, null);
-  }
-
-  /**
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param primitive STRING, INT64, ...
-   * @param name the name of the type
-   * @param length the length if the type is FIXED_LEN_BYTE_ARRAY, 0 otherwise
-   * @param originalType (optional) the original type (MAP, DECIMAL, UTF8, ...)
-   * @param decimalMeta (optional) metadata about the decimal type
-   * @param id the id of the field
-   */
-  PrimitiveType(
-      Repetition repetition, PrimitiveTypeName primitive,
-      int length, String name, OriginalType originalType,
-      DecimalMetadata decimalMeta, ID id) {
-    super(name, repetition, originalType, id);
-    this.primitive = primitive;
-    this.length = length;
-    this.decimalMeta = decimalMeta;
-  }
-
-  /**
-   * @param id the field id
-   * @return a new PrimitiveType with the same fields and a new id
-   */
-  @Override
-  public PrimitiveType withId(int id) {
-    return new PrimitiveType(getRepetition(), primitive, length, getName(), getOriginalType(), decimalMeta, new ID(id));
-  }
-
-  /**
-   * @return the primitive type
-   */
-  public PrimitiveTypeName getPrimitiveTypeName() {
-    return primitive;
-  }
-
-  /**
-   * @return the type length
-   */
-  public int getTypeLength() {
-    return length;
-  }
-
-  /**
-   * @return the decimal type metadata
-   */
-  public DecimalMetadata getDecimalMetadata() {
-    return decimalMeta;
-  }
-
-  /**
-   * @return true
-   */
-  @Override
-  public boolean isPrimitive() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void accept(TypeVisitor visitor) {
-    visitor.visit(this);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void writeToStringBuilder(StringBuilder sb, String indent) {
-    sb.append(indent)
-        .append(getRepetition().name().toLowerCase())
-        .append(" ")
-        .append(primitive.name().toLowerCase());
-    if (primitive == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
-      sb.append("(" + length + ")");
-    }
-    sb.append(" ").append(getName());
-    if (getOriginalType() != null) {
-      sb.append(" (").append(getOriginalType());
-      DecimalMetadata meta = getDecimalMetadata();
-      if (meta != null) {
-        sb.append("(")
-            .append(meta.getPrecision())
-            .append(",")
-            .append(meta.getScale())
-            .append(")");
-      }
-      sb.append(")");
-    }
-    if (getId() != null) {
-      sb.append(" = ").append(getId());
-    }
-  }
-
-  @Override @Deprecated
-  protected int typeHashCode() {
-    return hashCode();
-  }
-
-  @Override @Deprecated
-  protected boolean typeEquals(Type other) {
-    return equals(other);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  protected boolean equals(Type other) {
-    if (!other.isPrimitive()) {
-      return false;
-    }
-    PrimitiveType otherPrimitive = other.asPrimitiveType();
-    return super.equals(other)
-        && primitive == otherPrimitive.getPrimitiveTypeName()
-        && length == otherPrimitive.length
-        && eqOrBothNull(decimalMeta, otherPrimitive.decimalMeta);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public int hashCode() {
-    int hash = super.hashCode();
-    hash = hash * 31 + primitive.hashCode();
-    hash = hash * 31 + length;
-    if (decimalMeta != null) {
-      hash = hash * 31 + decimalMeta.hashCode();
-    }
-    return hash;
-  }
-
-  @Override
-  public int getMaxRepetitionLevel(String[] path, int i) {
-    if (path.length != i) {
-      throw new InvalidRecordException("Arrived at primitive node, path invalid");
-    }
-    return isRepetition(Repetition.REPEATED)? 1 : 0;
-  }
-
-  @Override
-  public int getMaxDefinitionLevel(String[] path, int i) {
-    if (path.length != i) {
-      throw new InvalidRecordException("Arrived at primitive node, path invalid");
-    }
-    return isRepetition(Repetition.REQUIRED) ? 0 : 1;
-  }
-
-  @Override
-  public Type getType(String[] path, int i) {
-    if (path.length != i) {
-      throw new InvalidRecordException("Arrived at primitive node at index " + i + " , path invalid: " + Arrays.toString(path));
-    }
-    return this;
-  }
-
-  @Override
-  protected List<String[]> getPaths(int depth) {
-    return Arrays.<String[]>asList(new String[depth]);
-  }
-
-  @Override
-  void checkContains(Type subType) {
-    super.checkContains(subType);
-    if (!subType.isPrimitive()) {
-      throw new InvalidRecordException(subType + " found: expected " + this);
-    }
-    PrimitiveType primitiveType = subType.asPrimitiveType();
-    if (this.primitive != primitiveType.primitive) {
-      throw new InvalidRecordException(subType + " found: expected " + this);
-    }
-
-  }
-
-  @Override
-  public <T> T convert(List<GroupType> path, TypeConverter<T> converter) {
-    return converter.convertPrimitiveType(path, this);
-  }
-
-  @Override
-  protected boolean containsPath(String[] path, int depth) {
-    return path.length == depth;
-  }
-
-  @Override
-  protected Type union(Type toMerge) {
-    return union(toMerge, true);
-  }
-
-  @Override
-  protected Type union(Type toMerge, boolean strict) {
-    if (!toMerge.isPrimitive() || (strict && !primitive.equals(toMerge.asPrimitiveType().getPrimitiveTypeName()))) {
-      throw new IncompatibleSchemaModificationException("can not merge type " + toMerge + " into " + this);
-    }
-    Types.PrimitiveBuilder<PrimitiveType> builder = Types.primitive(
-        primitive, toMerge.getRepetition());
-    if (PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY == primitive) {
-      builder.length(length);
-    }
-    return builder.named(getName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/Type.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/Type.java b/parquet-column/src/main/java/parquet/schema/Type.java
deleted file mode 100644
index cd72fa9..0000000
--- a/parquet-column/src/main/java/parquet/schema/Type.java
+++ /dev/null
@@ -1,317 +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 parquet.schema;
-
-import static parquet.Preconditions.checkNotNull;
-
-import java.util.List;
-
-import parquet.io.InvalidRecordException;
-
-/**
- * Represents the declared type for a field in a schema.
- * The Type object represents both the actual underlying type of the object
- * (eg a primitive or group) as well as its attributes such as whether it is
- * repeated, required, or optional.
- */
-abstract public class Type {
-
-  /**
-   * represents a field ID
-   *
-   * @author Julien Le Dem
-   *
-   */
-  public static final class ID {
-    private final int id;
-
-    public ID(int id) {
-      this.id = id;
-    }
-
-    public int intValue() {
-      return id;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return (obj instanceof ID) && ((ID)obj).id == id;
-    }
-
-    @Override
-    public int hashCode() {
-      return id;
-    }
-
-    @Override
-    public String toString() {
-      return String.valueOf(id);
-    }
-  }
-
-  /**
-   * Constraint on the repetition of a field
-   *
-   * @author Julien Le Dem
-   */
-  public static enum Repetition {
-    /**
-     * exactly 1
-     */
-    REQUIRED {
-      @Override
-      public boolean isMoreRestrictiveThan(Repetition other) {
-        return other != REQUIRED;
-      }
-    },
-    /**
-     * 0 or 1
-     */
-    OPTIONAL {
-      @Override
-      public boolean isMoreRestrictiveThan(Repetition other) {
-        return other == REPEATED;
-      }
-    },
-    /**
-     * 0 or more
-     */
-    REPEATED {
-      @Override
-      public boolean isMoreRestrictiveThan(Repetition other) {
-        return false;
-      }
-    }
-    ;
-
-    /**
-     * @param other
-     * @return true if it is strictly more restrictive than other
-     */
-    abstract public boolean isMoreRestrictiveThan(Repetition other);
-
-  }
-
-  private final String name;
-  private final Repetition repetition;
-  private final OriginalType originalType;
-  private final ID id;
-
-  /**
-   * @param name the name of the type
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   */
-  @Deprecated
-  public Type(String name, Repetition repetition) {
-    this(name, repetition, null, null);
-  }
-
-  /**
-   * @param name the name of the type
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param originalType (optional) the original type to help with cross schema conversion (LIST, MAP, ...)
-   */
-  @Deprecated
-  public Type(String name, Repetition repetition, OriginalType originalType) {
-    this(name, repetition, originalType, null);
-  }
-
-  /**
-   * @param name the name of the type
-   * @param repetition OPTIONAL, REPEATED, REQUIRED
-   * @param originalType (optional) the original type to help with cross schema conversion (LIST, MAP, ...)
-   * @param id (optional) the id of the fields.
-   */
-  Type(String name, Repetition repetition, OriginalType originalType, ID id) {
-    super();
-    this.name = checkNotNull(name, "name");
-    this.repetition = checkNotNull(repetition, "repetition");
-    this.originalType = originalType;
-    this.id = id;
-  }
-
-  /**
-   * @param id
-   * @return the same type with the id field set
-   */
-  public abstract Type withId(int id);
-
-  /**
-   * @return the name of the type
-   */
-  public String getName() {
-    return name;
-  }
-
-  /**
-   * @param rep
-   * @return if repetition of the type is rep
-   */
-  public boolean isRepetition(Repetition rep) {
-    return repetition == rep;
-  }
-
-  /**
-   * @return the repetition constraint
-   */
-  public Repetition getRepetition() {
-    return repetition;
-  }
-
-  /**
-   * @return the id of the field (if defined)
-   */
-  public ID getId() {
-    return id;
-  }
-
-  /**
-   * @return the original type (LIST, MAP, ...)
-   */
-  public OriginalType getOriginalType() {
-    return originalType;
-  }
-
-  /**
-   * @return if this is a primitive type
-   */
-  abstract public boolean isPrimitive();
-
-  /**
-   * @return this if it's a group type
-   * @throws ClassCastException if not
-   */
-  public GroupType asGroupType() {
-    if (isPrimitive()) {
-      throw new ClassCastException(this + " is not a group");
-    }
-    return (GroupType)this;
-  }
-
-  /**
-   * @return this if it's a primitive type
-   * @throws ClassCastException if not
-   */
-  public PrimitiveType asPrimitiveType() {
-    if (!isPrimitive()) {
-      throw new ClassCastException(this + " is not primitive");
-    }
-    return (PrimitiveType)this;
-  }
-
-  /**
-   * Writes a string representation to the provided StringBuilder
-   * @param sb the StringBuilder to write itself to
-   * @param indent indentation level
-   */
-  abstract public void writeToStringBuilder(StringBuilder sb, String indent);
-
-  /**
-   * Visits this type with the given visitor
-   * @param visitor the visitor to visit this type
-   */
-  abstract public void accept(TypeVisitor visitor);
-
-  @Deprecated
-  abstract protected int typeHashCode();
-
-  @Deprecated
-  abstract protected boolean typeEquals(Type other);
-
-  @Override
-  public int hashCode() {
-    int c = repetition.hashCode();
-    c = 31 * c + name.hashCode();
-    if (originalType != null) {
-      c = 31 * c +  originalType.hashCode();
-    }
-    if (id != null) {
-      c = 31 * c + id.hashCode();
-    }
-    return c;
-  }
-
-  protected boolean equals(Type other) {
-    return
-        name.equals(other.name)
-        && repetition == other.repetition
-        && eqOrBothNull(repetition, other.repetition)
-        && eqOrBothNull(id, other.id);
-  };
-
-  @Override
-  public boolean equals(Object other) {
-    if (!(other instanceof Type) || other == null) {
-      return false;
-    }
-    return equals((Type)other);
-  }
-
-  protected boolean eqOrBothNull(Object o1, Object o2) {
-    return (o1 == null && o2 == null) || (o1 != null && o1.equals(o2));
-  }
-
-  protected abstract int getMaxRepetitionLevel(String[] path, int i);
-
-  protected abstract int getMaxDefinitionLevel(String[] path, int i);
-
-  protected abstract Type getType(String[] path, int i);
-
-  protected abstract List<String[]> getPaths(int depth);
-
-  protected abstract boolean containsPath(String[] path, int depth);
-
-  /**
-   * @param toMerge the type to merge into this one
-   * @return the union result of merging toMerge into this
-   */
-  protected abstract Type union(Type toMerge);
-
-  /**
-   * @param toMerge the type to merge into this one
-   * @param strict should schema primitive types match
-   * @return the union result of merging toMerge into this
-   */
-  protected abstract Type union(Type toMerge, boolean strict);
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    writeToStringBuilder(sb, "");
-    return sb.toString();
-  }
-
-  void checkContains(Type subType) {
-    if (!this.name.equals(subType.name)
-        || this.repetition != subType.repetition) {
-      throw new InvalidRecordException(subType + " found: expected " + this);
-    }
-  }
-
-  /**
-   *
-   * @param converter logic to convert the tree
-   * @return the converted tree
-   */
-   abstract <T> T convert(List<GroupType> path, TypeConverter<T> converter);
-
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/TypeConverter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/TypeConverter.java b/parquet-column/src/main/java/parquet/schema/TypeConverter.java
deleted file mode 100644
index b3552a7..0000000
--- a/parquet-column/src/main/java/parquet/schema/TypeConverter.java
+++ /dev/null
@@ -1,55 +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 parquet.schema;
-
-import java.util.List;
-
-/**
- * to convert a MessageType tree
- * @see Type#convert(TypeConverter)
- *
- * @author Julien Le Dem
- *
- * @param <T> the resulting Type
- */
-public interface TypeConverter<T> {
-
-  /**
-   * @param path the path to that node
-   * @param primitiveType the type to convert
-   * @return the result of conversion
-   */
-  T convertPrimitiveType(List<GroupType> path, PrimitiveType primitiveType);
-
-  /**
-   * @param path the path to that node
-   * @param groupType the type to convert
-   * @param children its children already converted
-   * @return the result of conversion
-   */
-  T convertGroupType(List<GroupType> path, GroupType groupType, List<T> children);
-
-  /**
-   * @param messageType the type to convert
-   * @param children its children already converted
-   * @return the result of conversion
-   */
-  T convertMessageType(MessageType messageType, List<T> children);
-
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-column/src/main/java/parquet/schema/TypeVisitor.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/parquet/schema/TypeVisitor.java b/parquet-column/src/main/java/parquet/schema/TypeVisitor.java
deleted file mode 100644
index b2fae46..0000000
--- a/parquet-column/src/main/java/parquet/schema/TypeVisitor.java
+++ /dev/null
@@ -1,46 +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 parquet.schema;
-
-/**
- * Implement this interface to visit a schema
- *
- * type.accept(new TypeVisitor() { ... });
- *
- * @author Julien Le Dem
- *
- */
-public interface TypeVisitor {
-
-  /**
-   * @param groupType the group type to visit
-   */
-  void visit(GroupType groupType);
-
-  /**
-   * @param messageType the message type to visit
-   */
-  void visit(MessageType messageType);
-
-  /**
-   * @param primitiveType the primitive type to visit
-   */
-  void visit(PrimitiveType primitiveType);
-
-}