You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/11/23 18:54:40 UTC

svn commit: r1641233 [1/2] - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/io/parquet/convert/ test/org/apache/hadoop/hive/ql/io/parquet/ test/queries/clientpositive/ test/results/clientpositive/

Author: brock
Date: Sun Nov 23 17:54:38 2014
New Revision: 1641233

URL: http://svn.apache.org/r1641233
Log:
HIVE-8909 - Hive doesn't correctly read Parquet nested types (Ryan Blue via Brock)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ConverterParent.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestArrayCompatibility.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestMapStructures.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetDirect.java
    hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_multi_field_struct.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_optional_elements.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_required_elements.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_single_field_struct.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_structs.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_groups.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_primitives.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_primitives.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_single_field_struct.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_nested_complex.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_thrift_array_of_primitives.q
    hive/trunk/ql/src/test/queries/clientpositive/parquet_thrift_array_of_single_field_struct.q
    hive/trunk/ql/src/test/results/clientpositive/parquet_array_of_multi_field_struct.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_array_of_optional_elements.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_array_of_required_elements.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_array_of_single_field_struct.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_array_of_structs.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_array_of_unannotated_groups.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_array_of_unannotated_primitives.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_avro_array_of_primitives.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_avro_array_of_single_field_struct.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_nested_complex.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_thrift_array_of_primitives.q.out
    hive/trunk/ql/src/test/results/clientpositive/parquet_thrift_array_of_single_field_struct.q.out
Removed:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ArrayWritableGroupConverter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableGroupConverter.java
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ConverterParent.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ConverterParent.java?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ConverterParent.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ConverterParent.java Sun Nov 23 17:54:38 2014
@@ -0,0 +1,7 @@
+package org.apache.hadoop.hive.ql.io.parquet.convert;
+
+import org.apache.hadoop.io.Writable;
+
+interface ConverterParent {
+  void set(int index, Writable value);
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java?rev=1641233&r1=1641232&r2=1641233&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java Sun Nov 23 17:54:38 2014
@@ -13,11 +13,7 @@
  */
 package org.apache.hadoop.hive.ql.io.parquet.convert;
 
-import java.util.List;
-
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.ArrayWritable;
-
 import parquet.io.api.GroupConverter;
 import parquet.io.api.RecordMaterializer;
 import parquet.schema.GroupType;
@@ -29,10 +25,10 @@ import parquet.schema.GroupType;
  */
 public class DataWritableRecordConverter extends RecordMaterializer<ArrayWritable> {
 
-  private final DataWritableGroupConverter root;
+  private final HiveStructConverter root;
 
   public DataWritableRecordConverter(final GroupType requestedSchema, final GroupType tableSchema) {
-    this.root = new DataWritableGroupConverter(requestedSchema, tableSchema);
+    this.root = new HiveStructConverter(requestedSchema, tableSchema);
   }
 
   @Override
@@ -44,4 +40,4 @@ public class DataWritableRecordConverter
   public GroupConverter getRootConverter() {
     return root;
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java?rev=1641233&r1=1641232&r2=1641233&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java Sun Nov 23 17:54:38 2014
@@ -47,7 +47,7 @@ public enum ETypeConverter {
   EDOUBLE_CONVERTER(Double.TYPE) {
     @Override
 
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new PrimitiveConverter() {
         @Override
         public void addDouble(final double value) {
@@ -58,7 +58,7 @@ public enum ETypeConverter {
   },
   EBOOLEAN_CONVERTER(Boolean.TYPE) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new PrimitiveConverter() {
         @Override
         public void addBoolean(final boolean value) {
@@ -69,7 +69,7 @@ public enum ETypeConverter {
   },
   EFLOAT_CONVERTER(Float.TYPE) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new PrimitiveConverter() {
         @Override
         public void addFloat(final float value) {
@@ -80,7 +80,7 @@ public enum ETypeConverter {
   },
   EINT32_CONVERTER(Integer.TYPE) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new PrimitiveConverter() {
         @Override
         public void addInt(final int value) {
@@ -91,7 +91,7 @@ public enum ETypeConverter {
   },
   EINT64_CONVERTER(Long.TYPE) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new PrimitiveConverter() {
         @Override
         public void addLong(final long value) {
@@ -102,7 +102,7 @@ public enum ETypeConverter {
   },
   EBINARY_CONVERTER(Binary.class) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new BinaryConverter<BytesWritable>(type, parent, index) {
         @Override
         protected BytesWritable convert(Binary binary) {
@@ -113,7 +113,7 @@ public enum ETypeConverter {
   },
   ESTRING_CONVERTER(String.class) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new BinaryConverter<Text>(type, parent, index) {
         @Override
         protected Text convert(Binary binary) {
@@ -124,7 +124,7 @@ public enum ETypeConverter {
   },
   EDECIMAL_CONVERTER(BigDecimal.class) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new BinaryConverter<HiveDecimalWritable>(type, parent, index) {
         @Override
         protected HiveDecimalWritable convert(Binary binary) {
@@ -135,7 +135,7 @@ public enum ETypeConverter {
   },
   ETIMESTAMP_CONVERTER(TimestampWritable.class) {
     @Override
-    Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
       return new BinaryConverter<TimestampWritable>(type, parent, index) {
         @Override
         protected TimestampWritable convert(Binary binary) {
@@ -157,10 +157,10 @@ public enum ETypeConverter {
     return _type;
   }
 
-  abstract Converter getConverter(final PrimitiveType type, final int index, final HiveGroupConverter parent);
+  abstract PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent);
 
-  public static Converter getNewConverter(final PrimitiveType type, final int index,
-      final HiveGroupConverter parent) {
+  public static PrimitiveConverter getNewConverter(final PrimitiveType type, final int index,
+      final ConverterParent parent) {
     if (type.isPrimitive() && (type.asPrimitiveType().getPrimitiveTypeName().equals(PrimitiveType.PrimitiveTypeName.INT96))) {
       //TODO- cleanup once parquet support Timestamp type annotation.
       return ETypeConverter.ETIMESTAMP_CONVERTER.getConverter(type, index, parent);
@@ -183,11 +183,11 @@ public enum ETypeConverter {
 
   public abstract static class BinaryConverter<T extends Writable> extends PrimitiveConverter {
     protected final PrimitiveType type;
-    private final HiveGroupConverter parent;
+    private final ConverterParent parent;
     private final int index;
     private ArrayList<T> lookupTable;
 
-    public BinaryConverter(PrimitiveType type, HiveGroupConverter parent, int index) {
+    public BinaryConverter(PrimitiveType type, ConverterParent parent, int index) {
       this.type = type;
       this.parent = parent;
       this.index = index;

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java Sun Nov 23 17:54:38 2014
@@ -0,0 +1,164 @@
+package org.apache.hadoop.hive.ql.io.parquet.convert;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+import parquet.io.api.Converter;
+import parquet.schema.GroupType;
+import parquet.schema.Type;
+
+public class HiveCollectionConverter extends HiveGroupConverter {
+  private final GroupType collectionType;
+  private final ConverterParent parent;
+  private final int index;
+  private final Converter innerConverter;
+  private final List<Writable> list = new ArrayList<Writable>();
+
+  public static HiveGroupConverter forMap(GroupType mapType,
+                                          ConverterParent parent,
+                                          int index) {
+    return new HiveCollectionConverter(
+        mapType, parent, index, true /* its a map */ );
+  }
+
+  public static HiveGroupConverter forList(GroupType listType,
+                                           ConverterParent parent,
+                                           int index) {
+    return new HiveCollectionConverter(
+        listType, parent, index, false /* not a map */ );
+  }
+
+  private HiveCollectionConverter(GroupType collectionType,
+                                  ConverterParent parent,
+                                  int index, boolean isMap) {
+    this.collectionType = collectionType;
+    this.parent = parent;
+    this.index = index;
+    Type repeatedType = collectionType.getType(0);
+    if (isMap) {
+      this.innerConverter = new KeyValueConverter(
+          repeatedType.asGroupType(), this);
+    } else if (isElementType(repeatedType, collectionType.getName())) {
+      this.innerConverter = getConverterFromDescription(repeatedType, 0, this);
+    } else {
+      this.innerConverter = new ElementConverter(
+          repeatedType.asGroupType(), this);
+    }
+  }
+
+  @Override
+  public Converter getConverter(int fieldIndex) {
+    Preconditions.checkArgument(
+        fieldIndex == 0, "Invalid field index: " + fieldIndex);
+    return innerConverter;
+  }
+
+  @Override
+  public void start() {
+    list.clear();
+  }
+
+  @Override
+  public void end() {
+    parent.set(index, wrapList(new ArrayWritable(
+        Writable.class, list.toArray(new Writable[list.size()]))));
+  }
+
+  @Override
+  public void set(int index, Writable value) {
+    list.add(value);
+  }
+
+  private static class KeyValueConverter extends HiveGroupConverter {
+    private final HiveGroupConverter parent;
+    private final Converter keyConverter;
+    private final Converter valueConverter;
+    private Writable[] keyValue = null;
+
+    public KeyValueConverter(GroupType keyValueType, HiveGroupConverter parent) {
+      this.parent = parent;
+      this.keyConverter = getConverterFromDescription(
+          keyValueType.getType(0), 0, this);
+      this.valueConverter = getConverterFromDescription(
+          keyValueType.getType(1), 1, this);
+    }
+
+    @Override
+    public void set(int fieldIndex, Writable value) {
+      keyValue[fieldIndex] = value;
+    }
+
+    @Override
+    public Converter getConverter(int fieldIndex) {
+      switch (fieldIndex) {
+        case 0:
+          return keyConverter;
+        case 1:
+          return valueConverter;
+        default:
+          throw new IllegalArgumentException(
+              "Invalid field index for map key-value: " + fieldIndex);
+      }
+    }
+
+    @Override
+    public void start() {
+      this.keyValue = new Writable[2];
+    }
+
+    @Override
+    public void end() {
+      parent.set(0, new ArrayWritable(Writable.class, keyValue));
+    }
+  }
+
+  private static class ElementConverter extends HiveGroupConverter {
+    private final HiveGroupConverter parent;
+    private final Converter elementConverter;
+    private Writable element = null;
+
+    public ElementConverter(GroupType repeatedType, HiveGroupConverter parent) {
+      this.parent = parent;
+      this.elementConverter = getConverterFromDescription(
+          repeatedType.getType(0), 0, this);
+    }
+
+    @Override
+    public void set(int index, Writable value) {
+      this.element = value;
+    }
+
+    @Override
+    public Converter getConverter(int i) {
+      return elementConverter;
+    }
+
+    @Override
+    public void start() {
+      this.element = null;
+    }
+
+    @Override
+    public void end() {
+      parent.set(0, element);
+    }
+  }
+
+  private static boolean isElementType(Type repeatedType, String parentName) {
+    if (repeatedType.isPrimitive() ||
+        (repeatedType.asGroupType().getFieldCount() != 1)) {
+      return true;
+    } else if (repeatedType.getName().equals("array")) {
+      return true; // existing avro data
+    } else if (repeatedType.getName().equals(parentName + "_tuple")) {
+      return true; // existing thrift data
+    }
+    // false for the following cases:
+    // * name is "list", which matches the spec
+    // * name is "bag", which indicates existing hive or pig data
+    // * ambiguous case, which should be assumed is 3-level according to spec
+    return false;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java?rev=1641233&r1=1641232&r2=1641233&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java Sun Nov 23 17:54:38 2014
@@ -13,33 +13,62 @@
  */
 package org.apache.hadoop.hive.ql.io.parquet.convert;
 
+import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.Writable;
-
 import parquet.io.api.Converter;
 import parquet.io.api.GroupConverter;
+import parquet.io.api.PrimitiveConverter;
+import parquet.schema.GroupType;
+import parquet.schema.OriginalType;
+import parquet.schema.PrimitiveType;
 import parquet.schema.Type;
-import parquet.schema.Type.Repetition;
 
-public abstract class HiveGroupConverter extends GroupConverter {
+public abstract class HiveGroupConverter extends GroupConverter implements ConverterParent {
+
+  protected static PrimitiveConverter getConverterFromDescription(PrimitiveType type, int index, ConverterParent parent) {
+    if (type == null) {
+      return null;
+    }
+
+    return ETypeConverter.getNewConverter(type, index, parent);
+  }
+
+  protected static HiveGroupConverter getConverterFromDescription(GroupType type, int index, ConverterParent parent) {
+    if (type == null) {
+      return null;
+    }
+
+    OriginalType annotation = type.getOriginalType();
+    if (annotation == OriginalType.LIST) {
+      return HiveCollectionConverter.forList(type, parent, index);
+    } else if (annotation == OriginalType.MAP) {
+      return HiveCollectionConverter.forMap(type, parent, index);
+    }
 
-  protected static Converter getConverterFromDescription(final Type type, final int index,
-      final HiveGroupConverter parent) {
+    return new HiveStructConverter(type, parent, index);
+  }
+
+  protected static Converter getConverterFromDescription(Type type, int index, ConverterParent parent) {
     if (type == null) {
       return null;
     }
+
     if (type.isPrimitive()) {
-      return ETypeConverter.getNewConverter(type.asPrimitiveType(), index, parent);
-    } else {
-      if (type.asGroupType().getRepetition() == Repetition.REPEATED) {
-        return new ArrayWritableGroupConverter(type.asGroupType(), parent, index);
-      } else {
-        return new DataWritableGroupConverter(type.asGroupType(), parent, index);
-      }
+      return getConverterFromDescription(type.asPrimitiveType(), index, parent);
     }
+
+    return getConverterFromDescription(type.asGroupType(), index, parent);
   }
 
-  protected abstract void set(int index, Writable value);
+  /**
+   * The original list and map conversion didn't remove the synthetic layer and
+   * the ObjectInspector had to remove it. This is a temporary fix that adds an
+   * extra layer for the ObjectInspector to remove.
+   */
+  static ArrayWritable wrapList(ArrayWritable list) {
+    return new ArrayWritable(Writable.class, new Writable[] {list});
+  }
 
-  protected abstract void add(int index, Writable value);
+  public abstract void set(int index, Writable value);
 
-}
\ No newline at end of file
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java Sun Nov 23 17:54:38 2014
@@ -0,0 +1,131 @@
+/**
+ * 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.hadoop.hive.ql.io.parquet.convert;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+import parquet.io.api.Converter;
+import parquet.schema.GroupType;
+import parquet.schema.Type;
+
+/**
+ *
+ * A MapWritableGroupConverter, real converter between hive and parquet types recursively for complex types.
+ *
+ */
+public class HiveStructConverter extends HiveGroupConverter {
+
+  private final int totalFieldCount;
+  private final Converter[] converters;
+  private final ConverterParent parent;
+  private final int index;
+  private Writable[] writables;
+  private final List<Repeated> repeatedConverters;
+  private boolean reuseWritableArray = false;
+
+  public HiveStructConverter(final GroupType requestedSchema, final GroupType tableSchema) {
+    this(requestedSchema, null, 0, tableSchema);
+    this.reuseWritableArray = true;
+    this.writables = new Writable[tableSchema.getFieldCount()];
+  }
+
+  public HiveStructConverter(final GroupType groupType, final ConverterParent parent,
+                             final int index) {
+    this(groupType, parent, index, groupType);
+  }
+
+  public HiveStructConverter(final GroupType selectedGroupType,
+                             final ConverterParent parent, final int index, final GroupType containingGroupType) {
+    this.parent = parent;
+    this.index = index;
+    this.totalFieldCount = containingGroupType.getFieldCount();
+    final int selectedFieldCount = selectedGroupType.getFieldCount();
+
+    converters = new Converter[selectedFieldCount];
+    this.repeatedConverters = new ArrayList<Repeated>();
+
+    List<Type> selectedFields = selectedGroupType.getFields();
+    for (int i = 0; i < selectedFieldCount; i++) {
+      Type subtype = selectedFields.get(i);
+      if (containingGroupType.getFields().contains(subtype)) {
+        int fieldIndex = containingGroupType.getFieldIndex(subtype.getName());
+        converters[i] = getFieldConverter(subtype, fieldIndex);
+      } else {
+        throw new IllegalStateException("Group type [" + containingGroupType +
+            "] does not contain requested field: " + subtype);
+      }
+    }
+  }
+
+  private Converter getFieldConverter(Type type, int fieldIndex) {
+    Converter converter;
+    if (type.isRepetition(Type.Repetition.REPEATED)) {
+      if (type.isPrimitive()) {
+        converter = new Repeated.RepeatedPrimitiveConverter(
+            type.asPrimitiveType(), this, fieldIndex);
+      } else {
+        converter = new Repeated.RepeatedGroupConverter(
+            type.asGroupType(), this, fieldIndex);
+      }
+
+      repeatedConverters.add((Repeated) converter);
+    } else {
+      converter = getConverterFromDescription(type, fieldIndex, this);
+    }
+
+    return converter;
+  }
+
+  public final ArrayWritable getCurrentArray() {
+    return new ArrayWritable(Writable.class, writables);
+  }
+
+  @Override
+  public void set(int fieldIndex, Writable value) {
+    writables[fieldIndex] = value;
+  }
+
+  @Override
+  public Converter getConverter(final int fieldIndex) {
+    return converters[fieldIndex];
+  }
+
+  @Override
+  public void start() {
+    if (reuseWritableArray) {
+      // reset the array to null values
+      for (int i = 0; i < writables.length; i += 1) {
+        writables[i] = null;
+      }
+    } else {
+      this.writables = new Writable[totalFieldCount];
+    }
+    for (Repeated repeated : repeatedConverters) {
+      repeated.parentStart();
+    }
+  }
+
+  @Override
+  public void end() {
+    for (Repeated repeated : repeatedConverters) {
+      repeated.parentEnd();
+    }
+    if (parent != null) {
+      parent.set(index, getCurrentArray());
+    }
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java Sun Nov 23 17:54:38 2014
@@ -0,0 +1,155 @@
+package org.apache.hadoop.hive.ql.io.parquet.convert;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+import parquet.column.Dictionary;
+import parquet.io.api.Binary;
+import parquet.io.api.Converter;
+import parquet.io.api.PrimitiveConverter;
+import parquet.schema.GroupType;
+import parquet.schema.PrimitiveType;
+
+/**
+ * Converters for repeated fields need to know when the parent field starts and
+ * ends to correctly build lists from the repeated values.
+ */
+public interface Repeated extends ConverterParent {
+
+  public void parentStart();
+
+  public void parentEnd();
+
+  /**
+   * Stands in for a PrimitiveConverter and accumulates multiple values as an
+   * ArrayWritable.
+   */
+  class RepeatedPrimitiveConverter extends PrimitiveConverter implements Repeated {
+    private final PrimitiveType primitiveType;
+    private final PrimitiveConverter wrapped;
+    private final ConverterParent parent;
+    private final int index;
+    private final List<Writable> list = new ArrayList<Writable>();
+
+    public RepeatedPrimitiveConverter(PrimitiveType primitiveType, ConverterParent parent, int index) {
+      this.primitiveType = primitiveType;
+      this.parent = parent;
+      this.index = index;
+      this.wrapped = HiveGroupConverter.getConverterFromDescription(primitiveType, 0, this);
+    }
+
+    @Override
+    public boolean hasDictionarySupport() {
+      return wrapped.hasDictionarySupport();
+    }
+
+    @Override
+    public void setDictionary(Dictionary dictionary) {
+      wrapped.setDictionary(dictionary);
+    }
+
+    @Override
+    public void addValueFromDictionary(int dictionaryId) {
+      wrapped.addValueFromDictionary(dictionaryId);
+    }
+
+    @Override
+    public void addBinary(Binary value) {
+      wrapped.addBinary(value);
+    }
+
+    @Override
+    public void addBoolean(boolean value) {
+      wrapped.addBoolean(value);
+    }
+
+    @Override
+    public void addDouble(double value) {
+      wrapped.addDouble(value);
+    }
+
+    @Override
+    public void addFloat(float value) {
+      wrapped.addFloat(value);
+    }
+
+    @Override
+    public void addInt(int value) {
+      wrapped.addInt(value);
+    }
+
+    @Override
+    public void addLong(long value) {
+      wrapped.addLong(value);
+    }
+
+    @Override
+    public void parentStart() {
+      list.clear();
+    }
+
+    @Override
+    public void parentEnd() {
+      parent.set(index, HiveGroupConverter.wrapList(new ArrayWritable(
+          Writable.class, list.toArray(new Writable[list.size()]))));
+    }
+
+    @Override
+    public void set(int index, Writable value) {
+      list.add(value);
+    }
+  }
+
+  /**
+   * Stands in for a HiveGroupConverter and accumulates multiple values as an
+   * ArrayWritable.
+   */
+  class RepeatedGroupConverter extends HiveGroupConverter
+      implements Repeated {
+    private final GroupType groupType;
+    private final HiveGroupConverter wrapped;
+    private final ConverterParent parent;
+    private final int index;
+    private final List<Writable> list = new ArrayList<Writable>();
+
+    public RepeatedGroupConverter(GroupType groupType, ConverterParent parent, int index) {
+      this.groupType = groupType;
+      this.parent = parent;
+      this.index = index;
+      this.wrapped = HiveGroupConverter.getConverterFromDescription(groupType, 0, this);
+    }
+
+    @Override
+    public void set(int fieldIndex, Writable value) {
+      list.add(value);
+    }
+
+    @Override
+    public Converter getConverter(int fieldIndex) {
+      // delegate to the group's converters
+      return wrapped.getConverter(fieldIndex);
+    }
+
+    @Override
+    public void start() {
+      wrapped.start();
+    }
+
+    @Override
+    public void end() {
+      wrapped.end();
+    }
+
+    @Override
+    public void parentStart() {
+      list.clear();
+    }
+
+    @Override
+    public void parentEnd() {
+      parent.set(index, wrapList(new ArrayWritable(
+          Writable.class, list.toArray(new Writable[list.size()]))));
+    }
+  }
+}

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestArrayCompatibility.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestArrayCompatibility.java?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestArrayCompatibility.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestArrayCompatibility.java Sun Nov 23 17:54:38 2014
@@ -0,0 +1,614 @@
+package org.apache.hadoop.hive.ql.io.parquet;
+
+import java.util.List;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Assert;
+import org.junit.Test;
+import parquet.io.api.RecordConsumer;
+import parquet.schema.MessageType;
+import parquet.schema.Types;
+
+import static parquet.schema.OriginalType.LIST;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+
+public class TestArrayCompatibility extends TestParquetDirect {
+
+  @Test
+  public void testUnannotatedListOfPrimitives() throws Exception {
+    MessageType fileSchema = Types.buildMessage()
+        .repeated(INT32).named("list_of_ints")
+        .named("UnannotatedListOfPrimitives");
+
+    Path test = writeDirect("UnannotatedListOfPrimitives",
+        fileSchema,
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("list_of_ints", 0);
+
+            rc.addInteger(34);
+            rc.addInteger(35);
+            rc.addInteger(36);
+
+            rc.endField("list_of_ints", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        new IntWritable(34), new IntWritable(35), new IntWritable(36)));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testUnannotatedListOfGroups() throws Exception {
+    Path test = writeDirect("UnannotatedListOfGroups",
+        Types.buildMessage()
+            .repeatedGroup()
+                .required(FLOAT).named("x")
+                .required(FLOAT).named("y")
+                .named("list_of_points")
+            .named("UnannotatedListOfGroups"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("list_of_points", 0);
+
+            rc.startGroup();
+            rc.startField("x", 0);
+            rc.addFloat(1.0f);
+            rc.endField("x", 0);
+            rc.startField("y", 1);
+            rc.addFloat(1.0f);
+            rc.endField("y", 1);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("x", 0);
+            rc.addFloat(2.0f);
+            rc.endField("x", 0);
+            rc.startField("y", 1);
+            rc.addFloat(2.0f);
+            rc.endField("y", 1);
+            rc.endGroup();
+
+            rc.endField("list_of_points", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new FloatWritable(1.0f), new FloatWritable(1.0f)),
+        record(new FloatWritable(2.0f), new FloatWritable(2.0f))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testThriftPrimitiveInList() throws Exception {
+    Path test = writeDirect("ThriftPrimitiveInList",
+        Types.buildMessage()
+            .requiredGroup().as(LIST)
+                .repeated(INT32).named("list_of_ints_tuple")
+                .named("list_of_ints")
+            .named("ThriftPrimitiveInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("list_of_ints", 0);
+
+            rc.startGroup();
+            rc.startField("list_of_ints_tuple", 0);
+
+            rc.addInteger(34);
+            rc.addInteger(35);
+            rc.addInteger(36);
+
+            rc.endField("list_of_ints_tuple", 0);
+            rc.endGroup();
+
+            rc.endField("list_of_ints", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        new IntWritable(34), new IntWritable(35), new IntWritable(36)));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record", expected, records.get(0));
+  }
+
+  @Test
+  public void testThriftSingleFieldGroupInList() throws Exception {
+    // this tests the case where older data has an ambiguous structure, but the
+    // correct interpretation can be determined from the repeated name
+
+    Path test = writeDirect("ThriftSingleFieldGroupInList",
+        Types.buildMessage()
+            .optionalGroup().as(LIST)
+                .repeatedGroup()
+                    .required(INT64).named("count")
+                    .named("single_element_groups_tuple")
+                .named("single_element_groups")
+            .named("ThriftSingleFieldGroupInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("single_element_groups", 0);
+
+            rc.startGroup();
+            rc.startField("single_element_groups_tuple", 0); // start writing array contents
+
+            rc.startGroup();
+            rc.startField("count", 0);
+            rc.addLong(1234L);
+            rc.endField("count", 0);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("count", 0);
+            rc.addLong(2345L);
+            rc.endField("count", 0);
+            rc.endGroup();
+
+            rc.endField("single_element_groups_tuple", 0); // finished writing array contents
+            rc.endGroup();
+
+            rc.endField("single_element_groups", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new LongWritable(1234L)),
+        record(new LongWritable(2345L))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testAvroPrimitiveInList() throws Exception {
+    Path test = writeDirect("AvroPrimitiveInList",
+        Types.buildMessage()
+            .requiredGroup().as(LIST)
+                .repeated(INT32).named("array")
+                .named("list_of_ints")
+            .named("AvroPrimitiveInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("list_of_ints", 0);
+
+            rc.startGroup();
+            rc.startField("array", 0);
+
+            rc.addInteger(34);
+            rc.addInteger(35);
+            rc.addInteger(36);
+
+            rc.endField("array", 0);
+            rc.endGroup();
+
+            rc.endField("list_of_ints", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        new IntWritable(34), new IntWritable(35), new IntWritable(36)));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record", expected, records.get(0));
+  }
+
+  @Test
+  public void testAvroSingleFieldGroupInList() throws Exception {
+    // this tests the case where older data has an ambiguous structure, but the
+    // correct interpretation can be determined from the repeated name, "array"
+
+    Path test = writeDirect("AvroSingleFieldGroupInList",
+        Types.buildMessage()
+            .optionalGroup().as(LIST)
+                .repeatedGroup()
+                    .required(INT64).named("count")
+                    .named("array")
+                .named("single_element_groups")
+            .named("AvroSingleFieldGroupInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("single_element_groups", 0);
+
+            rc.startGroup();
+            rc.startField("array", 0); // start writing array contents
+
+            rc.startGroup();
+            rc.startField("count", 0);
+            rc.addLong(1234L);
+            rc.endField("count", 0);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("count", 0);
+            rc.addLong(2345L);
+            rc.endField("count", 0);
+            rc.endGroup();
+
+            rc.endField("array", 0); // finished writing array contents
+            rc.endGroup();
+
+            rc.endField("single_element_groups", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new LongWritable(1234L)),
+        record(new LongWritable(2345L))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testAmbiguousSingleFieldGroupInList() throws Exception {
+    // this tests the case where older data has an ambiguous list and is not
+    // named indicating that the source considered the group significant
+
+    Path test = writeDirect("SingleFieldGroupInList",
+        Types.buildMessage()
+            .optionalGroup().as(LIST)
+                .repeatedGroup()
+                    .required(INT64).named("count")
+                    .named("single_element_group")
+                .named("single_element_groups")
+            .named("SingleFieldGroupInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("single_element_groups", 0);
+
+            rc.startGroup();
+            rc.startField("single_element_group", 0); // start writing array contents
+
+            rc.startGroup();
+            rc.startField("count", 0);
+            rc.addLong(1234L);
+            rc.endField("count", 0);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("count", 0);
+            rc.addLong(2345L);
+            rc.endField("count", 0);
+            rc.endGroup();
+
+            rc.endField("single_element_group", 0); // finished writing array contents
+            rc.endGroup();
+
+            rc.endField("single_element_groups", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        new LongWritable(1234L),
+        new LongWritable(2345L)));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testMultiFieldGroupInList() throws Exception {
+    // tests the missing element layer, detected by a multi-field group
+
+    Path test = writeDirect("MultiFieldGroupInList",
+        Types.buildMessage()
+            .optionalGroup().as(LIST)
+                .repeatedGroup()
+                    .required(DOUBLE).named("latitude")
+                    .required(DOUBLE).named("longitude")
+                    .named("element") // should not affect schema conversion
+                .named("locations")
+            .named("MultiFieldGroupInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("locations", 0);
+
+            rc.startGroup();
+            rc.startField("element", 0);
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(0.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(180.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.endField("element", 0);
+            rc.endGroup();
+
+            rc.endField("locations", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new DoubleWritable(0.0), new DoubleWritable(0.0)),
+        record(new DoubleWritable(0.0), new DoubleWritable(180.0))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testNewOptionalGroupInList() throws Exception {
+    Path test = writeDirect("NewOptionalGroupInList",
+        Types.buildMessage()
+            .optionalGroup().as(LIST)
+                .repeatedGroup()
+                    .optionalGroup()
+                        .required(DOUBLE).named("latitude")
+                        .required(DOUBLE).named("longitude")
+                        .named("element")
+                    .named("list")
+                .named("locations")
+            .named("NewOptionalGroupInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("locations", 0);
+
+            rc.startGroup();
+            rc.startField("list", 0); // start writing array contents
+
+            // write a non-null element
+            rc.startGroup(); // array level
+            rc.startField("element", 0);
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(0.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.endField("element", 0);
+            rc.endGroup(); // array level
+
+            // write a null element (element field is omitted)
+            rc.startGroup(); // array level
+            rc.endGroup(); // array level
+
+            // write a second non-null element
+            rc.startGroup(); // array level
+            rc.startField("element", 0);
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(180.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.endField("element", 0);
+            rc.endGroup(); // array level
+
+            rc.endField("list", 0); // finished writing array contents
+            rc.endGroup();
+
+            rc.endField("locations", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new DoubleWritable(0.0), new DoubleWritable(0.0)),
+        null,
+        record(new DoubleWritable(0.0), new DoubleWritable(180.0))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testNewRequiredGroupInList() throws Exception {
+    Path test = writeDirect("NewRequiredGroupInList",
+        Types.buildMessage()
+            .optionalGroup().as(LIST)
+                .repeatedGroup()
+                    .requiredGroup()
+                        .required(DOUBLE).named("latitude")
+                        .required(DOUBLE).named("longitude")
+                        .named("element")
+                    .named("list")
+                .named("locations")
+            .named("NewRequiredGroupInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("locations", 0);
+
+            rc.startGroup();
+            rc.startField("list", 0); // start writing array contents
+
+            // write a non-null element
+            rc.startGroup(); // array level
+            rc.startField("element", 0);
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(180.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.endField("element", 0);
+            rc.endGroup(); // array level
+
+            // write a second non-null element
+            rc.startGroup(); // array level
+            rc.startField("element", 0);
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(0.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.endField("element", 0);
+            rc.endGroup(); // array level
+
+            rc.endField("list", 0); // finished writing array contents
+            rc.endGroup();
+
+            rc.endField("locations", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new DoubleWritable(0.0), new DoubleWritable(180.0)),
+        record(new DoubleWritable(0.0), new DoubleWritable(0.0))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+  @Test
+  public void testHiveRequiredGroupInList() throws Exception {
+    // this matches the list structure that Hive writes
+    Path test = writeDirect("HiveRequiredGroupInList",
+        Types.buildMessage()
+            .optionalGroup().as(LIST)
+                .repeatedGroup()
+                    .requiredGroup()
+                        .required(DOUBLE).named("latitude")
+                        .required(DOUBLE).named("longitude")
+                        .named("element")
+                    .named("bag")
+                .named("locations")
+            .named("HiveRequiredGroupInList"),
+        new DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("locations", 0);
+
+            rc.startGroup();
+            rc.startField("bag", 0); // start writing array contents
+
+            // write a non-null element
+            rc.startGroup(); // array level
+            rc.startField("element", 0);
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(180.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.endField("element", 0);
+            rc.endGroup(); // array level
+
+            // write a second non-null element
+            rc.startGroup(); // array level
+            rc.startField("element", 0);
+
+            rc.startGroup();
+            rc.startField("latitude", 0);
+            rc.addDouble(0.0);
+            rc.endField("latitude", 0);
+            rc.startField("longitude", 1);
+            rc.addDouble(0.0);
+            rc.endField("longitude", 1);
+            rc.endGroup();
+
+            rc.endField("element", 0);
+            rc.endGroup(); // array level
+
+            rc.endField("bag", 0); // finished writing array contents
+            rc.endGroup();
+
+            rc.endField("locations", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new DoubleWritable(0.0), new DoubleWritable(180.0)),
+        record(new DoubleWritable(0.0), new DoubleWritable(0.0))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+  }
+
+}

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestMapStructures.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestMapStructures.java?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestMapStructures.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestMapStructures.java Sun Nov 23 17:54:38 2014
@@ -0,0 +1,544 @@
+package org.apache.hadoop.hive.ql.io.parquet;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+import parquet.io.api.Binary;
+import parquet.io.api.RecordConsumer;
+import parquet.schema.Types;
+
+import static parquet.schema.OriginalType.*;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.*;
+
+public class TestMapStructures extends TestParquetDirect {
+
+  @Test
+  public void testStringMapRequiredPrimitive() throws Exception {
+    Path test = writeDirect("StringMapRequiredPrimitive",
+        Types.buildMessage()
+            .optionalGroup().as(MAP)
+                .repeatedGroup()
+                    .required(BINARY).as(UTF8).named("key")
+                    .required(INT32).named("value")
+                    .named("key_value")
+                .named("votes")
+            .named("StringMapRequiredPrimitive"),
+        new TestArrayCompatibility.DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("votes", 0);
+
+            rc.startGroup();
+            rc.startField("key_value", 0);
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("lettuce"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addInteger(34);
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("cabbage"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addInteger(18);
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.endField("key_value", 0);
+            rc.endGroup();
+
+            rc.endField("votes", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new Text("lettuce"), new IntWritable(34)),
+        record(new Text("cabbage"), new IntWritable(18))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+
+    deserialize(records.get(0),
+        Arrays.asList("votes"),
+        Arrays.asList("map<string,int>"));
+  }
+
+  @Test
+  public void testStringMapOptionalPrimitive() throws Exception {
+    Path test = writeDirect("StringMapOptionalPrimitive",
+        Types.buildMessage()
+            .optionalGroup().as(MAP)
+                .repeatedGroup()
+                    .required(BINARY).as(UTF8).named("key")
+                    .optional(INT32).named("value")
+                    .named("key_value")
+                .named("votes")
+            .named("StringMapOptionalPrimitive"),
+        new TestArrayCompatibility.DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("votes", 0);
+
+            rc.startGroup();
+            rc.startField("key_value", 0);
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("lettuce"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addInteger(34);
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("kale"));
+            rc.endField("key", 0);
+            // no value for kale
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("cabbage"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addInteger(18);
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.endField("key_value", 0);
+            rc.endGroup();
+
+            rc.endField("votes", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new Text("lettuce"), new IntWritable(34)),
+        record(new Text("kale"), null),
+        record(new Text("cabbage"), new IntWritable(18))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+
+    deserialize(records.get(0),
+        Arrays.asList("votes"),
+        Arrays.asList("map<string,int>"));
+  }
+
+  @Test
+  public void testStringMapOfOptionalArray() throws Exception {
+    // tests a multimap structure
+
+    Path test = writeDirect("StringMapOfOptionalArray",
+        Types.buildMessage()
+            .optionalGroup().as(MAP)
+                .repeatedGroup()
+                    .required(BINARY).as(UTF8).named("key")
+                    .optionalGroup().as(LIST)
+                        .repeatedGroup()
+                            .optional(BINARY).as(UTF8).named("element")
+                            .named("list")
+                        .named("value")
+                    .named("key_value")
+                .named("examples")
+            .named("StringMapOfOptionalArray"),
+        new TestArrayCompatibility.DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("examples", 0);
+
+            rc.startGroup();
+            rc.startField("key_value", 0);
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("green"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.startGroup();
+            rc.startField("list", 0);
+            rc.startGroup();
+            rc.startField("element", 0);
+            rc.addBinary(Binary.fromString("lettuce"));
+            rc.endField("element", 0);
+            rc.endGroup();
+            rc.startGroup();
+            rc.startField("element", 0);
+            rc.addBinary(Binary.fromString("kale"));
+            rc.endField("element", 0);
+            rc.endGroup();
+            rc.startGroup();
+            // adds a null element
+            rc.endGroup();
+            rc.endField("list", 0);
+            rc.endGroup();
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("brown"));
+            rc.endField("key", 0);
+            // no values array
+            rc.endGroup();
+
+            rc.endField("key_value", 0);
+            rc.endGroup();
+
+            rc.endField("examples", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new Text("green"), list(new Text("lettuce"), new Text("kale"), null)),
+        record(new Text("brown"), null)));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+
+    deserialize(records.get(0),
+        Arrays.asList("examples"),
+        Arrays.asList("map<string,array<string>>"));
+  }
+
+  @Test
+  public void testStringMapOfOptionalIntArray() throws Exception {
+    // tests a multimap structure for PARQUET-26
+
+    Path test = writeDirect("StringMapOfOptionalIntArray",
+        Types.buildMessage()
+            .optionalGroup().as(MAP)
+                .repeatedGroup()
+                    .required(BINARY).as(UTF8).named("key")
+                        .optionalGroup().as(LIST)
+                            .repeatedGroup()
+                            .optional(INT32).named("element")
+                            .named("list")
+                        .named("value")
+                    .named("key_value")
+                .named("examples")
+            .named("StringMapOfOptionalIntArray"),
+        new TestArrayCompatibility.DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("examples", 0);
+
+            rc.startGroup();
+            rc.startField("key_value", 0);
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("low"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.startGroup();
+            rc.startField("list", 0);
+            rc.startGroup();
+            rc.startField("element", 0);
+            rc.addInteger(34);
+            rc.endField("element", 0);
+            rc.endGroup();
+            rc.startGroup();
+            rc.startField("element", 0);
+            rc.addInteger(35);
+            rc.endField("element", 0);
+            rc.endGroup();
+            rc.startGroup();
+            // adds a null element
+            rc.endGroup();
+            rc.endField("list", 0);
+            rc.endGroup();
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("high"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.startGroup();
+            rc.startField("list", 0);
+            rc.startGroup();
+            rc.startField("element", 0);
+            rc.addInteger(340);
+            rc.endField("element", 0);
+            rc.endGroup();
+            rc.startGroup();
+            rc.startField("element", 0);
+            rc.addInteger(360);
+            rc.endField("element", 0);
+            rc.endGroup();
+            rc.endField("list", 0);
+            rc.endGroup();
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.endField("key_value", 0);
+            rc.endGroup();
+
+            rc.endField("examples", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new Text("low"), list(new IntWritable(34), new IntWritable(35), null)),
+        record(new Text("high"), list(new IntWritable(340), new IntWritable(360)))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+
+    deserialize(records.get(0),
+        Arrays.asList("examples"),
+        Arrays.asList("map<string,array<int>>"));
+  }
+
+  @Test
+  public void testMapWithComplexKey() throws Exception {
+    Path test = writeDirect("MapWithComplexKey",
+        Types.buildMessage()
+            .optionalGroup().as(MAP)
+                .repeatedGroup()
+                    .requiredGroup()
+                        .required(INT32).named("x")
+                        .required(INT32).named("y")
+                        .named("key")
+                    .optional(DOUBLE).named("value")
+                    .named("key_value")
+                .named("matrix")
+            .named("MapWithComplexKey"),
+        new TestArrayCompatibility.DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("matrix", 0);
+
+            rc.startGroup();
+            rc.startField("key_value", 0);
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.startGroup();
+            rc.startField("x", 0);
+            rc.addInteger(7);
+            rc.endField("x", 0);
+            rc.startField("y", 1);
+            rc.addInteger(22);
+            rc.endField("y", 1);
+            rc.endGroup();
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addDouble(3.14);
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.endField("key_value", 0);
+            rc.endGroup();
+
+            rc.endField("matrix", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(record(
+        record(new IntWritable(7), new IntWritable(22)),
+        new DoubleWritable(3.14))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+
+    deserialize(records.get(0),
+        Arrays.asList("matrix"),
+        Arrays.asList("map<struct<x:int,y:int>,bigint>"));
+  }
+
+  @Test
+  public void testDoubleMapWithStructValue() throws Exception {
+    Path test = writeDirect("DoubleMapWithStructValue",
+        Types.buildMessage()
+            .optionalGroup().as(MAP)
+                .repeatedGroup()
+                    .optional(DOUBLE).named("key")
+                    .optionalGroup()
+                        .required(INT32).named("x")
+                        .required(INT32).named("y")
+                        .named("value")
+                    .named("key_value")
+                .named("approx")
+            .named("DoubleMapWithStructValue"),
+        new TestArrayCompatibility.DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("approx", 0);
+
+            rc.startGroup();
+            rc.startField("key_value", 0);
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addDouble(3.14);
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.startGroup();
+            rc.startField("x", 0);
+            rc.addInteger(7);
+            rc.endField("x", 0);
+            rc.startField("y", 1);
+            rc.addInteger(22);
+            rc.endField("y", 1);
+            rc.endGroup();
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.endField("key_value", 0);
+            rc.endGroup();
+
+            rc.endField("approx", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(record(
+        new DoubleWritable(3.14),
+        record(new IntWritable(7), new IntWritable(22)))));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+
+    deserialize(records.get(0),
+        Arrays.asList("approx"),
+        Arrays.asList("map<bigint,struct<x:int,y:int>>"));
+  }
+
+  @Test
+  public void testNestedMap() throws Exception {
+    Path test = writeDirect("DoubleMapWithStructValue",
+        Types.buildMessage()
+            .optionalGroup().as(MAP)
+                .repeatedGroup()
+                    .optional(BINARY).as(UTF8).named("key")
+                        .optionalGroup().as(MAP)
+                            .repeatedGroup()
+                                .optional(BINARY).as(UTF8).named("key")
+                                .required(INT32).named("value")
+                            .named("key_value")
+                        .named("value")
+                    .named("key_value")
+                .named("map_of_maps")
+            .named("NestedMap"),
+        new TestArrayCompatibility.DirectWriter() {
+          @Override
+          public void write(RecordConsumer rc) {
+            rc.startMessage();
+            rc.startField("map_of_maps", 0);
+
+            rc.startGroup();
+            rc.startField("key_value", 0);
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("a"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.startGroup();
+            rc.startField("key_value", 0);
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("b"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addInteger(1);
+            rc.endField("value", 1);
+            rc.endGroup();
+            rc.endField("key_value", 0);
+            rc.endGroup();
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("b"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.startGroup();
+            rc.startField("key_value", 0);
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("a"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addInteger(-1);
+            rc.endField("value", 1);
+            rc.endGroup();
+            rc.startGroup();
+            rc.startField("key", 0);
+            rc.addBinary(Binary.fromString("b"));
+            rc.endField("key", 0);
+            rc.startField("value", 1);
+            rc.addInteger(-2);
+            rc.endField("value", 1);
+            rc.endGroup();
+            rc.endField("key_value", 0);
+            rc.endGroup();
+            rc.endField("value", 1);
+            rc.endGroup();
+
+            rc.endField("key_value", 0);
+            rc.endGroup();
+
+            rc.endField("map_of_maps", 0);
+            rc.endMessage();
+          }
+        });
+
+    ArrayWritable expected = record(list(
+        record(new Text("a"), list(
+            record(new Text("b"), new IntWritable(1)))),
+        record(new Text("b"), list(
+            record(new Text("a"), new IntWritable(-1)),
+            record(new Text("b"), new IntWritable(-2))))
+    ));
+
+    List<ArrayWritable> records = read(test);
+    Assert.assertEquals("Should have only one record", 1, records.size());
+    assertEquals("Should match expected record",
+        expected, records.get(0));
+
+    deserialize(records.get(0),
+        Arrays.asList("map_of_maps"),
+        Arrays.asList("map<string,map<string,int>>"));
+  }
+}

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetDirect.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetDirect.java?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetDirect.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetDirect.java Sun Nov 23 17:54:38 2014
@@ -0,0 +1,154 @@
+package org.apache.hadoop.hive.ql.io.parquet;
+
+import com.google.common.base.Joiner;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import parquet.hadoop.ParquetWriter;
+import parquet.hadoop.api.WriteSupport;
+import parquet.io.api.RecordConsumer;
+import parquet.schema.MessageType;
+
+public class TestParquetDirect {
+
+  public static FileSystem localFS = null;
+
+  @BeforeClass
+  public static void initializeFS() throws IOException {
+    localFS = FileSystem.getLocal(new Configuration());
+  }
+
+  @Rule
+  public final TemporaryFolder tempDir = new TemporaryFolder();
+
+
+  public interface DirectWriter {
+    public void write(RecordConsumer consumer);
+  }
+
+  public static class DirectWriteSupport extends WriteSupport<Void> {
+    private RecordConsumer recordConsumer;
+    private final MessageType type;
+    private final DirectWriter writer;
+    private final Map<String, String> metadata;
+
+    private DirectWriteSupport(MessageType type, DirectWriter writer,
+                               Map<String, String> metadata) {
+      this.type = type;
+      this.writer = writer;
+      this.metadata = metadata;
+    }
+
+    @Override
+    public WriteContext init(Configuration configuration) {
+      return new WriteContext(type, metadata);
+    }
+
+    @Override
+    public void prepareForWrite(RecordConsumer recordConsumer) {
+      this.recordConsumer = recordConsumer;
+    }
+
+    @Override
+    public void write(Void record) {
+      writer.write(recordConsumer);
+    }
+  }
+
+  public Path writeDirect(String name, MessageType type, DirectWriter writer)
+      throws IOException {
+    File temp = tempDir.newFile(name + ".parquet");
+    temp.deleteOnExit();
+    temp.delete();
+
+    Path path = new Path(temp.getPath());
+
+    ParquetWriter<Void> parquetWriter = new ParquetWriter<Void>(path,
+        new DirectWriteSupport(type, writer, new HashMap<String, String>()));
+    parquetWriter.write(null);
+    parquetWriter.close();
+
+    return path;
+  }
+
+  public static ArrayWritable record(Writable... fields) {
+    return new ArrayWritable(Writable.class, fields);
+  }
+
+  public static ArrayWritable list(Writable... elements) {
+    // the ObjectInspector for array<?> and map<?, ?> expects an extra layer
+    return new ArrayWritable(ArrayWritable.class, new ArrayWritable[] {
+        new ArrayWritable(Writable.class, elements)
+    });
+  }
+
+  public static String toString(ArrayWritable arrayWritable) {
+    Writable[] writables = arrayWritable.get();
+    String[] strings = new String[writables.length];
+    for (int i = 0; i < writables.length; i += 1) {
+      if (writables[i] instanceof ArrayWritable) {
+        strings[i] = toString((ArrayWritable) writables[i]);
+      } else {
+        strings[i] = String.valueOf(writables[i]);
+      }
+    }
+    return Arrays.toString(strings);
+  }
+
+  public static void assertEquals(String message, ArrayWritable expected,
+                                  ArrayWritable actual) {
+    Assert.assertEquals(message, toString(expected), toString(actual));
+  }
+
+  public static List<ArrayWritable> read(Path parquetFile) throws IOException {
+    List<ArrayWritable> records = new ArrayList<ArrayWritable>();
+
+    RecordReader<Void, ArrayWritable> reader = new MapredParquetInputFormat().
+        getRecordReader(new FileSplit(
+                parquetFile, 0, fileLength(parquetFile), (String[]) null),
+            new JobConf(), null);
+
+    Void alwaysNull = reader.createKey();
+    ArrayWritable record = reader.createValue();
+    while (reader.next(alwaysNull, record)) {
+      records.add(record);
+      record = reader.createValue(); // a new value so the last isn't clobbered
+    }
+
+    return records;
+  }
+
+  public static long fileLength(Path localFile) throws IOException {
+    return localFS.getFileStatus(localFile).getLen();
+  }
+
+  private static final Joiner COMMA = Joiner.on(",");
+  public void deserialize(Writable record, List<String> columnNames,
+                          List<String> columnTypes) throws Exception {
+    ParquetHiveSerDe serde = new ParquetHiveSerDe();
+    Properties props = new Properties();
+    props.setProperty(serdeConstants.LIST_COLUMNS, COMMA.join(columnNames));
+    props.setProperty(serdeConstants.LIST_COLUMN_TYPES, COMMA.join(columnTypes));
+    serde.initialize(null, props);
+    serde.deserialize(record);
+  }
+}

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_multi_field_struct.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_multi_field_struct.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_multi_field_struct.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_multi_field_struct.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,26 @@
+-- this test creates a Parquet table with an array of multi-field structs
+
+CREATE TABLE parquet_array_of_multi_field_structs (
+    locations ARRAY<STRUCT<latitude: DOUBLE, longitude: DOUBLE>>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/MultiFieldGroupInList.parquet'
+OVERWRITE INTO TABLE parquet_array_of_multi_field_structs;
+
+SELECT * FROM parquet_array_of_multi_field_structs;
+
+DROP TABLE parquet_array_of_multi_field_structs;
+
+-- maps use the same writable structure, so validate that the data can be read
+-- as a map instead of an array of structs
+
+CREATE TABLE parquet_map_view_of_multi_field_structs (
+    locations MAP<DOUBLE, DOUBLE>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/MultiFieldGroupInList.parquet'
+OVERWRITE INTO TABLE parquet_map_view_of_multi_field_structs;
+
+SELECT * FROM parquet_map_view_of_multi_field_structs;
+
+DROP TABLE parquet_map_view_of_multi_field_structs;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_optional_elements.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_optional_elements.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_optional_elements.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_optional_elements.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,12 @@
+-- this test creates a Parquet table with an array of optional structs
+
+CREATE TABLE parquet_array_of_optional_elements (
+    locations ARRAY<STRUCT<latitude: DOUBLE, longitude: DOUBLE>>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/NewOptionalGroupInList.parquet'
+OVERWRITE INTO TABLE parquet_array_of_optional_elements;
+
+SELECT * FROM parquet_array_of_optional_elements;
+
+DROP TABLE parquet_array_of_optional_elements;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_required_elements.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_required_elements.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_required_elements.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_required_elements.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,12 @@
+-- this test creates a Parquet table with an array of structs
+
+CREATE TABLE parquet_array_of_required_elements (
+    locations ARRAY<STRUCT<latitude: DOUBLE, longitude: DOUBLE>>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/NewRequiredGroupInList.parquet'
+OVERWRITE INTO TABLE parquet_array_of_required_elements;
+
+SELECT * FROM parquet_array_of_required_elements;
+
+DROP TABLE parquet_array_of_required_elements;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_single_field_struct.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_single_field_struct.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_single_field_struct.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_single_field_struct.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,14 @@
+-- this test creates a Parquet table with an array of single-field structs
+-- that has an ambiguous Parquet schema that is assumed to be a list of bigints
+-- This is verifies compliance with the spec for this case.
+
+CREATE TABLE parquet_ambiguous_array_of_single_field_structs (
+    single_element_groups ARRAY<BIGINT>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/SingleFieldGroupInList.parquet'
+OVERWRITE INTO TABLE parquet_ambiguous_array_of_single_field_structs;
+
+SELECT * FROM parquet_ambiguous_array_of_single_field_structs;
+
+DROP TABLE parquet_ambiguous_array_of_single_field_structs;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_structs.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_structs.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_structs.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_structs.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,12 @@
+-- this test creates a Parquet table with an array of structs
+
+CREATE TABLE parquet_array_of_structs (
+    locations ARRAY<STRUCT<latitude: DOUBLE, longitude: DOUBLE>>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/HiveRequiredGroupInList.parquet'
+OVERWRITE INTO TABLE parquet_array_of_structs;
+
+SELECT * FROM parquet_array_of_structs;
+
+DROP TABLE parquet_array_of_structs;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_groups.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_groups.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_groups.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_groups.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,13 @@
+-- this test creates a Parquet table from a structure with an unannotated
+-- repeated structure of (x,y) structs
+
+CREATE TABLE parquet_array_of_unannotated_groups (
+    list_of_points ARRAY<STRUCT<x: FLOAT, y: FLOAT>>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/UnannotatedListOfGroups.parquet'
+OVERWRITE INTO TABLE parquet_array_of_unannotated_groups;
+
+SELECT * FROM parquet_array_of_unannotated_groups;
+
+DROP TABLE parquet_array_of_unannotated_groups;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_primitives.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_primitives.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_primitives.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_array_of_unannotated_primitives.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,13 @@
+-- this test creates a Parquet table from a structure with an unannotated
+-- repeated structure of int32s
+
+CREATE TABLE parquet_array_of_unannotated_ints (
+    list_of_ints ARRAY<INT>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/UnannotatedListOfPrimitives.parquet'
+OVERWRITE INTO TABLE parquet_array_of_unannotated_ints;
+
+SELECT * FROM parquet_array_of_unannotated_ints;
+
+DROP TABLE parquet_array_of_unannotated_ints;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_primitives.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_primitives.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_primitives.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_primitives.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,12 @@
+-- this test creates a Parquet table with an array of structs
+
+CREATE TABLE parquet_avro_array_of_primitives (
+    list_of_ints ARRAY<INT>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/AvroPrimitiveInList.parquet'
+OVERWRITE INTO TABLE parquet_avro_array_of_primitives;
+
+SELECT * FROM parquet_avro_array_of_primitives;
+
+DROP TABLE parquet_avro_array_of_primitives;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_single_field_struct.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_single_field_struct.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_single_field_struct.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_avro_array_of_single_field_struct.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,13 @@
+-- this test creates a Parquet table with an array of single-field structs
+-- as written by parquet-avro
+
+CREATE TABLE parquet_avro_array_of_single_field_structs (
+    single_element_groups ARRAY<STRUCT<count: BIGINT>>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/AvroSingleFieldGroupInList.parquet'
+OVERWRITE INTO TABLE parquet_avro_array_of_single_field_structs;
+
+SELECT * FROM parquet_avro_array_of_single_field_structs;
+
+DROP TABLE parquet_avro_array_of_single_field_structs;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_nested_complex.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_nested_complex.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_nested_complex.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_nested_complex.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,29 @@
+-- start with the original nestedcomplex test
+
+create table nestedcomplex (
+simple_int int,
+max_nested_array  array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<int>>>>>>>>>>>>>>>>>>>>>>>,
+max_nested_map    array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<map<string,string>>>>>>>>>>>>>>>>>>>>>>,
+max_nested_struct array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<struct<s:string, i:bigint>>>>>>>>>>>>>>>>>>>>>>>,
+simple_string string)
+ROW FORMAT SERDE
+   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+WITH SERDEPROPERTIES (
+   'hive.serialization.extend.nesting.levels'='true',
+   'line.delim'='\n'
+)
+;
+
+describe nestedcomplex;
+describe extended nestedcomplex;
+
+load data local inpath '../../data/files/nested_complex.txt' overwrite into table nestedcomplex;
+
+-- and load the table into Parquet
+
+CREATE TABLE parquet_nested_complex STORED AS PARQUET AS SELECT * FROM nestedcomplex;
+
+SELECT * FROM parquet_nested_complex SORT BY simple_int;
+
+DROP TABLE nestedcomplex;
+DROP TABLE parquet_nested_complex;

Added: hive/trunk/ql/src/test/queries/clientpositive/parquet_thrift_array_of_primitives.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/parquet_thrift_array_of_primitives.q?rev=1641233&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/parquet_thrift_array_of_primitives.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/parquet_thrift_array_of_primitives.q Sun Nov 23 17:54:38 2014
@@ -0,0 +1,12 @@
+-- this test creates a Parquet table with an array of structs
+
+CREATE TABLE parquet_thrift_array_of_primitives (
+    list_of_ints ARRAY<INT>
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/ThriftPrimitiveInList.parquet'
+OVERWRITE INTO TABLE parquet_thrift_array_of_primitives;
+
+SELECT * FROM parquet_thrift_array_of_primitives;
+
+DROP TABLE parquet_thrift_array_of_primitives;