[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] ORC nested Identity partition support

2020-05-22 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r429328821



##
File path: 
spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java
##
@@ -109,23 +104,8 @@
 Iterator iter;
 
 if (hasJoinedPartitionColumns) {
-  if (SUPPORTS_CONSTANTS.contains(file.format())) {
-iterSchema = requiredSchema;
-iter = open(task, requiredSchema, PartitionUtil.constantsMap(task, 
RowDataReader::convertConstant));
-  } else {
-// schema used to read data files
-Schema readSchema = TypeUtil.selectNot(requiredSchema, idColumns);
-Schema partitionSchema = TypeUtil.select(requiredSchema, idColumns);
-PartitionRowConverter convertToRow = new 
PartitionRowConverter(partitionSchema, spec);
-JoinedRow joined = new JoinedRow();
-
-InternalRow partition = convertToRow.apply(file.partition());
-joined.withRight(partition);
-
-// create joined rows and project from the joined schema to the final 
schema
-iterSchema = TypeUtil.join(readSchema, partitionSchema);
-iter = Iterators.transform(open(task, readSchema, ImmutableMap.of()), 
joined::withLeft);
-  }
+  iterSchema = requiredSchema;
+  iter = open(task, requiredSchema, PartitionUtil.constantsMap(task, 
RowDataReader::convertConstant));

Review comment:
   #1004 is in. That might be why this has conflicts.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] ORC nested Identity partition support

2020-05-22 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r429327828



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java
##
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+
+public class OrcValueReaders {
+  private OrcValueReaders() {
+  }
+
+  public static OrcValReader booleans() {
+return BooleanReader.INSTANCE;
+  }
+
+  public static OrcValReader shorts() {
+return ShortReader.INSTANCE;
+  }
+
+  public static OrcValReader ints() {
+return IntegerReader.INSTANCE;
+  }
+
+  public static OrcValReader longs() {
+return LongReader.INSTANCE;
+  }
+
+  public static OrcValReader floats() {
+return FloatReader.INSTANCE;
+  }
+
+  public static OrcValReader doubles() {
+return DoubleReader.INSTANCE;
+  }
+
+  public static OrcValReader bytes() {
+return BytesReader.INSTANCE;
+  }
+
+  public static OrcValReader byteReader() {
+return ByteReader.INSTANCE;
+  }
+
+  private static class BooleanReader implements OrcValReader {
+static final OrcValReader INSTANCE = new BooleanReader();
+
+private BooleanReader() {
+}
+
+@Override
+public Boolean nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row] != 0;
+}
+  }
+
+  private static class ShortReader implements OrcValReader {
+static final OrcValReader INSTANCE = new ShortReader();
+
+private ShortReader() {
+}
+
+@Override
+public Short nonNullRead(ColumnVector vector, int row) {
+  return (short) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class IntegerReader implements OrcValReader {
+static final OrcValReader INSTANCE = new IntegerReader();
+
+private IntegerReader() {
+}
+
+@Override
+public Integer nonNullRead(ColumnVector vector, int row) {
+  return (int) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class LongReader implements OrcValReader {
+static final OrcValReader INSTANCE = new LongReader();
+
+private LongReader() {
+}
+
+@Override
+public Long nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class FloatReader implements OrcValReader {
+private static final FloatReader INSTANCE = new FloatReader();
+
+private FloatReader() {
+}
+
+@Override
+public Float nonNullRead(ColumnVector vector, int row) {
+  return (float) ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class DoubleReader implements OrcValReader {
+private static final DoubleReader INSTANCE = new DoubleReader();
+
+private DoubleReader() {
+}
+
+@Override
+public Double nonNullRead(ColumnVector vector, int row) {
+  return ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class ByteReader implements OrcValReader {
+private static final ByteReader INSTANCE = new ByteReader();
+
+private ByteReader() {
+}
+
+@Override
+public Byte nonNullRead(ColumnVector vector, int row) {
+  return (byte) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class BytesReader implements OrcValReader {
+private static final BytesReader INSTANCE = new BytesReader();
+
+private BytesReader() {
+}
+
+@Override
+public byte[] nonNullRead(ColumnVector vector, int row) {
+  BytesColumnVector bytesVector = (BytesColumnVector) vector;
+
+  return Arrays.copyOfRange(
+  

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] ORC nested Identity partition support

2020-05-22 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r429326033



##
File path: orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
##
@@ -308,7 +309,7 @@ private static TypeDescription buildOrcProjection(Integer 
fieldId, Type type, bo
   orcType = convert(fieldId, type, false);
 }
 }
-
+orcType.setAttribute(ICEBERG_ID_ATTRIBUTE, fieldId.toString());

Review comment:
   It would be great to know what's going on here. Since this is just a 
projection schema and the reader is built with the Iceberg schema (that has 
required/optional), I don't think it is really a blocker. But setting a 
property here shouldn't cause ORC to fail, right?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421676500



##
File path: 
spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java
##
@@ -71,7 +71,7 @@ private void writeAndValidateRecords(Schema schema, 
Iterable expect
 
 try (CloseableIterable reader = 
ORC.read(Files.localInput(testFile))
 .project(schema)
-.createReaderFunc(SparkOrcReader::new)

Review comment:
   I think this should test with and without container reuse if that is 
implemented in this PR. Probably just make this test parameterized.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421674571



##
File path: 
spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java
##
@@ -109,23 +104,8 @@
 Iterator iter;
 
 if (hasJoinedPartitionColumns) {
-  if (SUPPORTS_CONSTANTS.contains(file.format())) {
-iterSchema = requiredSchema;
-iter = open(task, requiredSchema, PartitionUtil.constantsMap(task, 
RowDataReader::convertConstant));
-  } else {
-// schema used to read data files
-Schema readSchema = TypeUtil.selectNot(requiredSchema, idColumns);
-Schema partitionSchema = TypeUtil.select(requiredSchema, idColumns);
-PartitionRowConverter convertToRow = new 
PartitionRowConverter(partitionSchema, spec);
-JoinedRow joined = new JoinedRow();
-
-InternalRow partition = convertToRow.apply(file.partition());
-joined.withRight(partition);
-
-// create joined rows and project from the joined schema to the final 
schema
-iterSchema = TypeUtil.join(readSchema, partitionSchema);
-iter = Iterators.transform(open(task, readSchema, ImmutableMap.of()), 
joined::withLeft);
-  }
+  iterSchema = requiredSchema;
+  iter = open(task, requiredSchema, PartitionUtil.constantsMap(task, 
RowDataReader::convertConstant));

Review comment:
   Once this and #1004 are in, we can remove the Spark projection!





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421673301



##
File path: 
spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java
##
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.data;
+
+import com.google.common.collect.Lists;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.orc.OrcValReader;
+import org.apache.iceberg.orc.OrcValueReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.types.UTF8String;
+
+
+class SparkOrcValueReaders {
+  private SparkOrcValueReaders() {
+  }
+
+  static OrcValReader strings() {
+return StringReader.INSTANCE;
+  }
+
+  static OrcValReader timestampTzs() {
+return TimestampTzReader.INSTANCE;
+  }
+
+  static OrcValReader struct(
+  List> readers, Types.StructType struct, Map 
idToConstant) {
+return new StructReader(readers, struct, idToConstant);
+  }
+
+  static OrcValReader array(OrcValReader elementReader) {
+return new ArrayReader(elementReader);
+  }
+
+  static OrcValReader map(OrcValReader keyReader, OrcValReader 
valueReader) {
+return new MapReader(keyReader, valueReader);
+  }
+
+  private static class ArrayReader implements OrcValReader {
+private final OrcValReader elementReader;
+private final List reusedList = Lists.newArrayList();
+
+private ArrayReader(OrcValReader elementReader) {
+  this.elementReader = elementReader;
+}
+
+@Override
+public ArrayData nonNullRead(ColumnVector vector, int row) {
+  reusedList.clear();
+  ListColumnVector listVector = (ListColumnVector) vector;
+  int offset = (int) listVector.offsets[row];
+  int length = (int) listVector.lengths[row];
+  for (int c = 0; c < length; ++c) {
+reusedList.add(elementReader.read(listVector.child, offset + c));
+  }
+  return new GenericArrayData(reusedList.toArray());
+}
+  }
+
+  private static class MapReader implements OrcValReader {
+private final OrcValReader keyReader;
+private final OrcValReader valueReader;
+
+private final List reusedKeyList = Lists.newArrayList();
+private final List reusedValueList = Lists.newArrayList();
+
+private MapReader(OrcValReader keyReader, OrcValReader valueReader) {
+  this.keyReader = keyReader;
+  this.valueReader = valueReader;
+}
+
+@Override
+public MapData nonNullRead(ColumnVector vector, int row) {
+  reusedKeyList.clear();
+  reusedValueList.clear();
+  MapColumnVector mapVector = (MapColumnVector) vector;
+  int offset = (int) mapVector.offsets[row];
+  long length = mapVector.lengths[row];
+  for (int c = 0; c < length; c++) {
+reusedKeyList.add(keyReader.read(mapVector.keys, offset + c));
+reusedValueList.add(valueReader.read(mapVector.values, offset + c));
+  }
+
+  return new ArrayBasedMapData(
+  new GenericArrayData(reusedKeyList.toArray()),
+  new GenericArrayData(reusedValueList.toArray()));
+}
+  }
+
+  static class StructReader extends OrcValueReaders.StructReader {
+private final int numFields;
+private InternalRow internalRow;

Review comment:
   As I noted above, the reader can't keep 

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421672821



##
File path: 
spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java
##
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.data;
+
+import com.google.common.collect.Lists;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.orc.OrcValReader;
+import org.apache.iceberg.orc.OrcValueReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.types.UTF8String;
+
+
+class SparkOrcValueReaders {
+  private SparkOrcValueReaders() {
+  }
+
+  static OrcValReader strings() {
+return StringReader.INSTANCE;
+  }
+
+  static OrcValReader timestampTzs() {
+return TimestampTzReader.INSTANCE;
+  }
+
+  static OrcValReader struct(
+  List> readers, Types.StructType struct, Map 
idToConstant) {
+return new StructReader(readers, struct, idToConstant);
+  }
+
+  static OrcValReader array(OrcValReader elementReader) {
+return new ArrayReader(elementReader);
+  }
+
+  static OrcValReader map(OrcValReader keyReader, OrcValReader 
valueReader) {
+return new MapReader(keyReader, valueReader);
+  }
+
+  private static class ArrayReader implements OrcValReader {
+private final OrcValReader elementReader;
+private final List reusedList = Lists.newArrayList();
+
+private ArrayReader(OrcValReader elementReader) {
+  this.elementReader = elementReader;
+}
+
+@Override
+public ArrayData nonNullRead(ColumnVector vector, int row) {
+  reusedList.clear();
+  ListColumnVector listVector = (ListColumnVector) vector;
+  int offset = (int) listVector.offsets[row];
+  int length = (int) listVector.lengths[row];
+  for (int c = 0; c < length; ++c) {
+reusedList.add(elementReader.read(listVector.child, offset + c));
+  }
+  return new GenericArrayData(reusedList.toArray());
+}
+  }
+
+  private static class MapReader implements OrcValReader {
+private final OrcValReader keyReader;
+private final OrcValReader valueReader;
+
+private final List reusedKeyList = Lists.newArrayList();
+private final List reusedValueList = Lists.newArrayList();
+
+private MapReader(OrcValReader keyReader, OrcValReader valueReader) {
+  this.keyReader = keyReader;
+  this.valueReader = valueReader;
+}
+
+@Override
+public MapData nonNullRead(ColumnVector vector, int row) {
+  reusedKeyList.clear();
+  reusedValueList.clear();
+  MapColumnVector mapVector = (MapColumnVector) vector;
+  int offset = (int) mapVector.offsets[row];
+  long length = mapVector.lengths[row];
+  for (int c = 0; c < length; c++) {
+reusedKeyList.add(keyReader.read(mapVector.keys, offset + c));
+reusedValueList.add(valueReader.read(mapVector.values, offset + c));
+  }
+
+  return new ArrayBasedMapData(
+  new GenericArrayData(reusedKeyList.toArray()),
+  new GenericArrayData(reusedValueList.toArray()));
+}
+  }
+
+  static class StructReader extends OrcValueReaders.StructReader {
+private final int numFields;
+private final InternalRow internalRow;
+
+protected StructReader(List> readers, Types.StructType 

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421671989



##
File path: 
spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java
##
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.data;
+
+import com.google.common.collect.Lists;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.orc.OrcValReader;
+import org.apache.iceberg.orc.OrcValueReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.types.UTF8String;
+
+
+class SparkOrcValueReaders {
+  private SparkOrcValueReaders() {
+  }
+
+  static OrcValReader strings() {
+return StringReader.INSTANCE;
+  }
+
+  static OrcValReader timestampTzs() {
+return TimestampTzReader.INSTANCE;
+  }
+
+  static OrcValReader struct(
+  List> readers, Types.StructType struct, Map 
idToConstant) {
+return new StructReader(readers, struct, idToConstant);
+  }
+
+  static OrcValReader array(OrcValReader elementReader) {
+return new ArrayReader(elementReader);
+  }
+
+  static OrcValReader map(OrcValReader keyReader, OrcValReader 
valueReader) {
+return new MapReader(keyReader, valueReader);
+  }
+
+  private static class ArrayReader implements OrcValReader {
+private final OrcValReader elementReader;
+private final List reusedList = Lists.newArrayList();

Review comment:
   Container reuse requires passing objects into the read method for 
correctness.
   
   The contract for container reuse is that the caller will consume the record 
before asking for the next one. For example, Spark might copy the entire row 
into an UnsafeRow.
   
   The problem with keeping a reused list or struct in the reader itself is 
that the reader might be called more than once to produce a value before the 
row is consumed. For example, a list of structs `locations 
list>` will call the inner reader for each 
location struct in the list before returning the record. If the reader reuses 
the struct, then the same struct will be added to the list multiple times and 
all of them will have the last values set in the reused struct.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421668822



##
File path: 
spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java
##
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.data;
+
+import com.google.common.collect.Lists;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.orc.OrcValReader;
+import org.apache.iceberg.orc.OrcValueReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.types.UTF8String;
+
+
+class SparkOrcValueReaders {
+  private SparkOrcValueReaders() {
+  }
+
+  static OrcValReader strings() {

Review comment:
   Consider renaming this to something that makes it more obvious that it 
is returning `UTF8String`. Avro has a factory method named `utf8s` to 
distinguish between strings and its custom `Utf8` class.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421666462



##
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java
##
@@ -53,425 +40,83 @@
  * It minimizes allocations by reusing most of the objects in the 
implementation.
  */
 public class SparkOrcReader implements OrcValueReader {
-  private static final int INITIAL_SIZE = 128 * 1024;
-  private final List columns;
-  private final Converter[] converters;
-  private final UnsafeRowWriter rowWriter;
+  private final SparkOrcValueReaders.StructReader reader;
 
-  public SparkOrcReader(TypeDescription readOrcSchema) {
-columns = readOrcSchema.getChildren();
-converters = buildConverters();
-rowWriter = new UnsafeRowWriter(columns.size(), INITIAL_SIZE);
+  public SparkOrcReader(org.apache.iceberg.Schema expectedSchema, 
TypeDescription readSchema) {
+this(expectedSchema, readSchema, ImmutableMap.of());
   }
 
-  private Converter[] buildConverters() {
-final Converter[] newConverters = new Converter[columns.size()];
-for (int c = 0; c < newConverters.length; ++c) {
-  newConverters[c] = buildConverter(columns.get(c));
-}
-return newConverters;
+  @SuppressWarnings("unchecked")
+  public SparkOrcReader(
+  org.apache.iceberg.Schema expectedSchema, TypeDescription readOrcSchema, 
Map idToConstant) {
+reader = (SparkOrcValueReaders.StructReader) 
OrcSchemaWithTypeVisitor.visit(
+expectedSchema, readOrcSchema, new ReadBuilder(idToConstant));
   }
 
   @Override
   public InternalRow read(VectorizedRowBatch batch, int row) {
-rowWriter.reset();
-rowWriter.zeroOutNullBytes();
-for (int c = 0; c < batch.cols.length; ++c) {
-  converters[c].convert(rowWriter, c, batch.cols[c], row);
-}
-return rowWriter.getRow();
+return reader.read(batch, row);

Review comment:
   This is where you could wrap the columns from `VectorizedRowBatch` in a 
`StructColumnVector`.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421665947



##
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java
##
@@ -53,425 +40,83 @@
  * It minimizes allocations by reusing most of the objects in the 
implementation.
  */
 public class SparkOrcReader implements OrcValueReader {
-  private static final int INITIAL_SIZE = 128 * 1024;
-  private final List columns;
-  private final Converter[] converters;
-  private final UnsafeRowWriter rowWriter;
+  private final SparkOrcValueReaders.StructReader reader;
 
-  public SparkOrcReader(TypeDescription readOrcSchema) {
-columns = readOrcSchema.getChildren();
-converters = buildConverters();
-rowWriter = new UnsafeRowWriter(columns.size(), INITIAL_SIZE);
+  public SparkOrcReader(org.apache.iceberg.Schema expectedSchema, 
TypeDescription readSchema) {
+this(expectedSchema, readSchema, ImmutableMap.of());
   }
 
-  private Converter[] buildConverters() {
-final Converter[] newConverters = new Converter[columns.size()];
-for (int c = 0; c < newConverters.length; ++c) {
-  newConverters[c] = buildConverter(columns.get(c));
-}
-return newConverters;
+  @SuppressWarnings("unchecked")
+  public SparkOrcReader(
+  org.apache.iceberg.Schema expectedSchema, TypeDescription readOrcSchema, 
Map idToConstant) {
+reader = (SparkOrcValueReaders.StructReader) 
OrcSchemaWithTypeVisitor.visit(

Review comment:
   Minor: we prefer to use `this.reader =` when assigning to instance 
fields so it's clear that it is setting a field.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421665455



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java
##
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+
+public class OrcValueReaders {
+  private OrcValueReaders() {
+  }
+
+  public static OrcValReader booleans() {
+return BooleanReader.INSTANCE;
+  }
+
+  public static OrcValReader shorts() {
+return ShortReader.INSTANCE;
+  }
+
+  public static OrcValReader ints() {
+return IntegerReader.INSTANCE;
+  }
+
+  public static OrcValReader longs() {
+return LongReader.INSTANCE;
+  }
+
+  public static OrcValReader floats() {
+return FloatReader.INSTANCE;
+  }
+
+  public static OrcValReader doubles() {
+return DoubleReader.INSTANCE;
+  }
+
+  public static OrcValReader bytes() {
+return BytesReader.INSTANCE;
+  }
+
+  public static OrcValReader byteReader() {
+return ByteReader.INSTANCE;
+  }
+
+  private static class BooleanReader implements OrcValReader {
+static final OrcValReader INSTANCE = new BooleanReader();
+
+private BooleanReader() {
+}
+
+@Override
+public Boolean nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row] != 0;
+}
+  }
+
+  private static class ShortReader implements OrcValReader {
+static final OrcValReader INSTANCE = new ShortReader();
+
+private ShortReader() {
+}
+
+@Override
+public Short nonNullRead(ColumnVector vector, int row) {
+  return (short) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class IntegerReader implements OrcValReader {
+static final OrcValReader INSTANCE = new IntegerReader();
+
+private IntegerReader() {
+}
+
+@Override
+public Integer nonNullRead(ColumnVector vector, int row) {
+  return (int) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class LongReader implements OrcValReader {
+static final OrcValReader INSTANCE = new LongReader();
+
+private LongReader() {
+}
+
+@Override
+public Long nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class FloatReader implements OrcValReader {
+private static final FloatReader INSTANCE = new FloatReader();
+
+private FloatReader() {
+}
+
+@Override
+public Float nonNullRead(ColumnVector vector, int row) {
+  return (float) ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class DoubleReader implements OrcValReader {
+private static final DoubleReader INSTANCE = new DoubleReader();
+
+private DoubleReader() {
+}
+
+@Override
+public Double nonNullRead(ColumnVector vector, int row) {
+  return ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class ByteReader implements OrcValReader {
+private static final ByteReader INSTANCE = new ByteReader();
+
+private ByteReader() {
+}
+
+@Override
+public Byte nonNullRead(ColumnVector vector, int row) {
+  return (byte) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class BytesReader implements OrcValReader {
+private static final BytesReader INSTANCE = new BytesReader();
+
+private BytesReader() {
+}
+
+@Override
+public byte[] nonNullRead(ColumnVector vector, int row) {
+  BytesColumnVector bytesVector = (BytesColumnVector) vector;
+
+  return Arrays.copyOfRange(
+  

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421661929



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java
##
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+
+public class OrcValueReaders {
+  private OrcValueReaders() {
+  }
+
+  public static OrcValReader booleans() {
+return BooleanReader.INSTANCE;
+  }
+
+  public static OrcValReader shorts() {
+return ShortReader.INSTANCE;
+  }
+
+  public static OrcValReader ints() {
+return IntegerReader.INSTANCE;
+  }
+
+  public static OrcValReader longs() {
+return LongReader.INSTANCE;
+  }
+
+  public static OrcValReader floats() {
+return FloatReader.INSTANCE;
+  }
+
+  public static OrcValReader doubles() {
+return DoubleReader.INSTANCE;
+  }
+
+  public static OrcValReader bytes() {
+return BytesReader.INSTANCE;
+  }
+
+  public static OrcValReader byteReader() {
+return ByteReader.INSTANCE;
+  }
+
+  private static class BooleanReader implements OrcValReader {
+static final OrcValReader INSTANCE = new BooleanReader();
+
+private BooleanReader() {
+}
+
+@Override
+public Boolean nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row] != 0;
+}
+  }
+
+  private static class ShortReader implements OrcValReader {
+static final OrcValReader INSTANCE = new ShortReader();
+
+private ShortReader() {
+}
+
+@Override
+public Short nonNullRead(ColumnVector vector, int row) {
+  return (short) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class IntegerReader implements OrcValReader {
+static final OrcValReader INSTANCE = new IntegerReader();
+
+private IntegerReader() {
+}
+
+@Override
+public Integer nonNullRead(ColumnVector vector, int row) {
+  return (int) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class LongReader implements OrcValReader {
+static final OrcValReader INSTANCE = new LongReader();
+
+private LongReader() {
+}
+
+@Override
+public Long nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class FloatReader implements OrcValReader {
+private static final FloatReader INSTANCE = new FloatReader();
+
+private FloatReader() {
+}
+
+@Override
+public Float nonNullRead(ColumnVector vector, int row) {
+  return (float) ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class DoubleReader implements OrcValReader {
+private static final DoubleReader INSTANCE = new DoubleReader();
+
+private DoubleReader() {
+}
+
+@Override
+public Double nonNullRead(ColumnVector vector, int row) {
+  return ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class ByteReader implements OrcValReader {
+private static final ByteReader INSTANCE = new ByteReader();
+
+private ByteReader() {
+}
+
+@Override
+public Byte nonNullRead(ColumnVector vector, int row) {
+  return (byte) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class BytesReader implements OrcValReader {
+private static final BytesReader INSTANCE = new BytesReader();
+
+private BytesReader() {
+}
+
+@Override
+public byte[] nonNullRead(ColumnVector vector, int row) {
+  BytesColumnVector bytesVector = (BytesColumnVector) vector;
+
+  return Arrays.copyOfRange(
+  

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421658103



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java
##
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+
+public class OrcValueReaders {
+  private OrcValueReaders() {
+  }
+
+  public static OrcValReader booleans() {
+return BooleanReader.INSTANCE;
+  }
+
+  public static OrcValReader shorts() {
+return ShortReader.INSTANCE;
+  }
+
+  public static OrcValReader ints() {
+return IntegerReader.INSTANCE;
+  }
+
+  public static OrcValReader longs() {
+return LongReader.INSTANCE;
+  }
+
+  public static OrcValReader floats() {
+return FloatReader.INSTANCE;
+  }
+
+  public static OrcValReader doubles() {
+return DoubleReader.INSTANCE;
+  }
+
+  public static OrcValReader bytes() {
+return BytesReader.INSTANCE;
+  }
+
+  public static OrcValReader byteReader() {
+return ByteReader.INSTANCE;
+  }
+
+  private static class BooleanReader implements OrcValReader {
+static final OrcValReader INSTANCE = new BooleanReader();
+
+private BooleanReader() {
+}
+
+@Override
+public Boolean nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row] != 0;
+}
+  }
+
+  private static class ShortReader implements OrcValReader {
+static final OrcValReader INSTANCE = new ShortReader();
+
+private ShortReader() {
+}
+
+@Override
+public Short nonNullRead(ColumnVector vector, int row) {
+  return (short) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class IntegerReader implements OrcValReader {
+static final OrcValReader INSTANCE = new IntegerReader();
+
+private IntegerReader() {
+}
+
+@Override
+public Integer nonNullRead(ColumnVector vector, int row) {
+  return (int) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class LongReader implements OrcValReader {
+static final OrcValReader INSTANCE = new LongReader();
+
+private LongReader() {
+}
+
+@Override
+public Long nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class FloatReader implements OrcValReader {
+private static final FloatReader INSTANCE = new FloatReader();
+
+private FloatReader() {
+}
+
+@Override
+public Float nonNullRead(ColumnVector vector, int row) {
+  return (float) ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class DoubleReader implements OrcValReader {
+private static final DoubleReader INSTANCE = new DoubleReader();
+
+private DoubleReader() {
+}
+
+@Override
+public Double nonNullRead(ColumnVector vector, int row) {
+  return ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class ByteReader implements OrcValReader {
+private static final ByteReader INSTANCE = new ByteReader();
+
+private ByteReader() {
+}
+
+@Override
+public Byte nonNullRead(ColumnVector vector, int row) {
+  return (byte) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class BytesReader implements OrcValReader {
+private static final BytesReader INSTANCE = new BytesReader();
+
+private BytesReader() {
+}
+
+@Override
+public byte[] nonNullRead(ColumnVector vector, int row) {
+  BytesColumnVector bytesVector = (BytesColumnVector) vector;
+
+  return Arrays.copyOfRange(
+  

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421656784



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java
##
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+
+public class OrcValueReaders {
+  private OrcValueReaders() {
+  }
+
+  public static OrcValReader booleans() {
+return BooleanReader.INSTANCE;
+  }
+
+  public static OrcValReader shorts() {
+return ShortReader.INSTANCE;
+  }
+
+  public static OrcValReader ints() {
+return IntegerReader.INSTANCE;
+  }
+
+  public static OrcValReader longs() {
+return LongReader.INSTANCE;
+  }
+
+  public static OrcValReader floats() {
+return FloatReader.INSTANCE;
+  }
+
+  public static OrcValReader doubles() {
+return DoubleReader.INSTANCE;
+  }
+
+  public static OrcValReader bytes() {
+return BytesReader.INSTANCE;
+  }
+
+  public static OrcValReader byteReader() {
+return ByteReader.INSTANCE;
+  }
+
+  private static class BooleanReader implements OrcValReader {
+static final OrcValReader INSTANCE = new BooleanReader();
+
+private BooleanReader() {
+}
+
+@Override
+public Boolean nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row] != 0;
+}
+  }
+
+  private static class ShortReader implements OrcValReader {
+static final OrcValReader INSTANCE = new ShortReader();
+
+private ShortReader() {
+}
+
+@Override
+public Short nonNullRead(ColumnVector vector, int row) {
+  return (short) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class IntegerReader implements OrcValReader {
+static final OrcValReader INSTANCE = new IntegerReader();
+
+private IntegerReader() {
+}
+
+@Override
+public Integer nonNullRead(ColumnVector vector, int row) {
+  return (int) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class LongReader implements OrcValReader {
+static final OrcValReader INSTANCE = new LongReader();
+
+private LongReader() {
+}
+
+@Override
+public Long nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class FloatReader implements OrcValReader {
+private static final FloatReader INSTANCE = new FloatReader();
+
+private FloatReader() {
+}
+
+@Override
+public Float nonNullRead(ColumnVector vector, int row) {
+  return (float) ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class DoubleReader implements OrcValReader {
+private static final DoubleReader INSTANCE = new DoubleReader();
+
+private DoubleReader() {
+}
+
+@Override
+public Double nonNullRead(ColumnVector vector, int row) {
+  return ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class ByteReader implements OrcValReader {
+private static final ByteReader INSTANCE = new ByteReader();
+
+private ByteReader() {
+}
+
+@Override
+public Byte nonNullRead(ColumnVector vector, int row) {
+  return (byte) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class BytesReader implements OrcValReader {
+private static final BytesReader INSTANCE = new BytesReader();
+
+private BytesReader() {
+}
+
+@Override
+public byte[] nonNullRead(ColumnVector vector, int row) {
+  BytesColumnVector bytesVector = (BytesColumnVector) vector;
+
+  return Arrays.copyOfRange(
+  

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421655358



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java
##
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+
+public class OrcValueReaders {
+  private OrcValueReaders() {
+  }
+
+  public static OrcValReader booleans() {
+return BooleanReader.INSTANCE;
+  }
+
+  public static OrcValReader shorts() {
+return ShortReader.INSTANCE;
+  }
+
+  public static OrcValReader ints() {
+return IntegerReader.INSTANCE;
+  }
+
+  public static OrcValReader longs() {
+return LongReader.INSTANCE;
+  }
+
+  public static OrcValReader floats() {
+return FloatReader.INSTANCE;
+  }
+
+  public static OrcValReader doubles() {
+return DoubleReader.INSTANCE;
+  }
+
+  public static OrcValReader bytes() {
+return BytesReader.INSTANCE;
+  }
+
+  public static OrcValReader byteReader() {
+return ByteReader.INSTANCE;
+  }
+
+  private static class BooleanReader implements OrcValReader {
+static final OrcValReader INSTANCE = new BooleanReader();
+
+private BooleanReader() {
+}
+
+@Override
+public Boolean nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row] != 0;
+}
+  }
+
+  private static class ShortReader implements OrcValReader {
+static final OrcValReader INSTANCE = new ShortReader();
+
+private ShortReader() {
+}
+
+@Override
+public Short nonNullRead(ColumnVector vector, int row) {
+  return (short) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class IntegerReader implements OrcValReader {
+static final OrcValReader INSTANCE = new IntegerReader();
+
+private IntegerReader() {
+}
+
+@Override
+public Integer nonNullRead(ColumnVector vector, int row) {
+  return (int) ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class LongReader implements OrcValReader {
+static final OrcValReader INSTANCE = new LongReader();
+
+private LongReader() {
+}
+
+@Override
+public Long nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row];
+}
+  }
+
+  private static class FloatReader implements OrcValReader {
+private static final FloatReader INSTANCE = new FloatReader();
+
+private FloatReader() {
+}
+
+@Override
+public Float nonNullRead(ColumnVector vector, int row) {
+  return (float) ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class DoubleReader implements OrcValReader {
+private static final DoubleReader INSTANCE = new DoubleReader();
+
+private DoubleReader() {
+}
+
+@Override
+public Double nonNullRead(ColumnVector vector, int row) {
+  return ((DoubleColumnVector) vector).vector[row];
+}
+  }
+
+  private static class ByteReader implements OrcValReader {

Review comment:
   Similar to short, do we need this reader?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421654780



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java
##
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+
+public class OrcValueReaders {
+  private OrcValueReaders() {
+  }
+
+  public static OrcValReader booleans() {
+return BooleanReader.INSTANCE;
+  }
+
+  public static OrcValReader shorts() {
+return ShortReader.INSTANCE;
+  }
+
+  public static OrcValReader ints() {
+return IntegerReader.INSTANCE;
+  }
+
+  public static OrcValReader longs() {
+return LongReader.INSTANCE;
+  }
+
+  public static OrcValReader floats() {
+return FloatReader.INSTANCE;
+  }
+
+  public static OrcValReader doubles() {
+return DoubleReader.INSTANCE;
+  }
+
+  public static OrcValReader bytes() {
+return BytesReader.INSTANCE;
+  }
+
+  public static OrcValReader byteReader() {
+return ByteReader.INSTANCE;
+  }
+
+  private static class BooleanReader implements OrcValReader {
+static final OrcValReader INSTANCE = new BooleanReader();
+
+private BooleanReader() {
+}
+
+@Override
+public Boolean nonNullRead(ColumnVector vector, int row) {
+  return ((LongColumnVector) vector).vector[row] != 0;
+}
+  }
+
+  private static class ShortReader implements OrcValReader {

Review comment:
   Do we need a short reader? Iceberg doesn't support short values in a 
table schema, so we shouldn't be expecting to ever return a short value.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421653711



##
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValReader.java
##
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+
+
+public interface OrcValReader {

Review comment:
   That works for me.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-07 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r421650510



##
File path: 
orc/src/main/java/org/apache/iceberg/orc/OrcSchemaWithTypeVisitor.java
##
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.collect.Lists;
+import java.util.List;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+
+
+public abstract class OrcSchemaWithTypeVisitor {
+  public static  T visit(
+  org.apache.iceberg.Schema iSchema, TypeDescription schema, 
OrcSchemaWithTypeVisitor visitor) {
+return visit(iSchema.asStruct(), schema, visitor);
+  }
+
+  public static  T visit(Type iType, TypeDescription schema, 
OrcSchemaWithTypeVisitor visitor) {
+switch (schema.getCategory()) {
+  case STRUCT:
+return visitRecord(iType.asStructType(), schema, visitor);
+
+  case UNION:
+// We don't have an answer for union types.
+throw new IllegalArgumentException("Can't handle " + schema);
+
+  case LIST:
+Types.ListType list = iType.asListType();
+return visitor.array(

Review comment:
   In other visitors, we try to name the method after the type in the 
schema that is being visited. That's why Avro uses `array` but Iceberg uses 
`list`. Since the category for ORC is `LIST`, should the visitor method be 
named `list`?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-06 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r420997677



##
File path: orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
##
@@ -496,7 +503,7 @@ private static boolean isRequired(TypeDescription orcType) {
 }
   }
 
-  private static int getMaxIcebergId(TypeDescription originalOrcSchema) {
+  static int getMaxIcebergId(TypeDescription originalOrcSchema) {

Review comment:
   I don't see this used other than in the conversion logic to assign new 
IDs. Because that assignment is actually a correctness bug, we don't need this 
method at all. Also, since it isn't used anywhere else in this PR there is no 
need to make it package-private instead of private.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-06 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r420996718



##
File path: orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
##
@@ -214,6 +214,7 @@ public static Schema convert(TypeDescription orcSchema) {
 "Error in ORC file, children fields and names do not match.");
 
 List icebergFields = 
Lists.newArrayListWithExpectedSize(children.size());
+// TODO how we get field ids from ORC schema

Review comment:
   I just noticed the logic here and it's a correctness bug. ORC should not 
assign column IDs when one is missing. Instead, it should ignore the field.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #989: [WIP] Orc nested Identity partition support

2020-05-02 Thread GitBox


rdblue commented on a change in pull request #989:
URL: https://github.com/apache/incubator-iceberg/pull/989#discussion_r419017570



##
File path: 
spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java
##
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.data;
+
+import com.google.common.collect.Lists;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.orc.OrcValReader;
+import org.apache.iceberg.orc.OrcValueReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.types.UTF8String;
+
+
+class SparkOrcValueReaders {
+  private SparkOrcValueReaders() {
+  }
+
+  static OrcValReader strings() {
+return StringReader.INSTANCE;
+  }
+
+  static OrcValReader timestampTzs() {
+return TimestampTzReader.INSTANCE;
+  }
+
+  static OrcValReader struct(
+  List> readers, Types.StructType struct, Map 
idToConstant) {
+return new StructReader(readers, struct, idToConstant);
+  }
+
+  static OrcValReader array(OrcValReader elementReader) {
+return new ArrayReader(elementReader);
+  }
+
+  static OrcValReader map(OrcValReader keyReader, OrcValReader 
valueReader) {
+return new MapReader(keyReader, valueReader);
+  }
+
+  private static class ArrayReader implements OrcValReader {
+private final OrcValReader elementReader;
+private final List reusedList = Lists.newArrayList();
+
+private ArrayReader(OrcValReader elementReader) {
+  this.elementReader = elementReader;
+}
+
+@Override
+public ArrayData nonNullRead(ColumnVector vector, int row) {
+  reusedList.clear();
+  ListColumnVector listVector = (ListColumnVector) vector;
+  int offset = (int) listVector.offsets[row];
+  int length = (int) listVector.lengths[row];
+  for (int c = 0; c < length; ++c) {
+reusedList.add(elementReader.read(listVector.child, offset + c));
+  }
+  return new GenericArrayData(reusedList.toArray());
+}
+  }
+
+  private static class MapReader implements OrcValReader {
+private final OrcValReader keyReader;
+private final OrcValReader valueReader;
+
+private final List reusedKeyList = Lists.newArrayList();
+private final List reusedValueList = Lists.newArrayList();
+
+private MapReader(OrcValReader keyReader, OrcValReader valueReader) {
+  this.keyReader = keyReader;
+  this.valueReader = valueReader;
+}
+
+@Override
+public MapData nonNullRead(ColumnVector vector, int row) {
+  reusedKeyList.clear();
+  reusedValueList.clear();
+  MapColumnVector mapVector = (MapColumnVector) vector;
+  int offset = (int) mapVector.offsets[row];
+  long length = mapVector.lengths[row];
+  for (int c = 0; c < length; c++) {
+reusedKeyList.add(keyReader.read(mapVector.keys, offset + c));
+reusedValueList.add(valueReader.read(mapVector.values, offset + c));
+  }
+
+  return new ArrayBasedMapData(
+  new GenericArrayData(reusedKeyList.toArray()),
+  new GenericArrayData(reusedValueList.toArray()));
+}
+  }
+
+  static class StructReader extends OrcValueReaders.StructReader {
+private final int numFields;
+private final InternalRow internalRow;
+
+protected StructReader(List> readers, Types.StructType