rdblue commented on a change in pull request #139: ORC support integration for 
Spark 2.4.0
URL: https://github.com/apache/incubator-iceberg/pull/139#discussion_r270993148
 
 

 ##########
 File path: 
spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java
 ##########
 @@ -0,0 +1,823 @@
+/*
+ * Copyright 2018 Hortonworks
+ *
+ * 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.iceberg.spark.data;
+
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ColumnIdMap;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.orc.OrcIterator;
+import org.apache.iceberg.orc.TypeConversion;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.common.type.FastHiveDecimal;
+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.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.storage.serde2.io.DateWritable;
+import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.SpecializedGetters;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter;
+import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter;
+import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeWriter;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.Platform;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Converts the OrcInterator, which returns ORC's VectorizedRowBatch to a
+ * set of Spark's UnsafeRows.
+ *
+ * It minimizes allocations by reusing most of the objects in the 
implementation.
+ */
+public class SparkOrcReader implements Iterator<InternalRow>, Closeable {
+  private final static int INITIAL_SIZE = 128 * 1024;
+  private final OrcIterator reader;
+  private final UnsafeRowWriter writer;
+  private int nextRow = 0;
+  private VectorizedRowBatch current = null;
+  private Converter[] converter;
+
+  public SparkOrcReader(InputFile location,
+                        FileScanTask task,
+                        Schema readSchema) {
+    ColumnIdMap columnIds = new ColumnIdMap();
+    TypeDescription orcSchema = TypeConversion.toOrc(readSchema, columnIds);
+    reader = ORC.read(location)
+        .split(task.start(), task.length())
+        .schema(readSchema)
+        .build();
+    int numFields = readSchema.columns().size();
+    writer = new UnsafeRowWriter(numFields, INITIAL_SIZE);
+    converter = new Converter[numFields];
+    for(int c = 0; c < numFields; ++c) {
+      converter[c] = buildConverter(writer, orcSchema.getChildren().get(c));
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    return (current != null && nextRow < current.size) || reader.hasNext();
+  }
+
+  @Override
+  public UnsafeRow next() {
+    if (current == null || nextRow >= current.size) {
+      current = reader.next();
+      nextRow = 0;
+    }
+
+    writer.reset();
+    writer.zeroOutNullBytes();
+    for(int c=0; c < current.cols.length; ++c) {
+      converter[c].convert(writer, c, current.cols[c], nextRow);
+    }
+    nextRow++;
+    return writer.getRow();
+  }
+
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+
+  private static void printRow(SpecializedGetters row, TypeDescription schema) 
{
+    List<TypeDescription> children = schema.getChildren();
+    System.out.print("{");
+    for(int c = 0; c < children.size(); ++c) {
+      System.out.print("\"" + schema.getFieldNames().get(c) + "\": ");
+      printRow(row, c, children.get(c));
+    }
+    System.out.print("}");
+  }
+
+  private static void printRow(SpecializedGetters row, int ord, 
TypeDescription schema) {
+    switch (schema.getCategory()) {
+      case BOOLEAN:
+        System.out.print(row.getBoolean(ord));
+        break;
+      case BYTE:
+        System.out.print(row.getByte(ord));
+        break;
+      case SHORT:
+        System.out.print(row.getShort(ord));
+        break;
+      case INT:
+        System.out.print(row.getInt(ord));
+        break;
+      case LONG:
+        System.out.print(row.getLong(ord));
+        break;
+      case FLOAT:
+        System.out.print(row.getFloat(ord));
+        break;
+      case DOUBLE:
+        System.out.print(row.getDouble(ord));
+        break;
+      case CHAR:
+      case VARCHAR:
+      case STRING:
+        System.out.print("\"" + row.getUTF8String(ord) + "\"");
+        break;
+      case BINARY: {
+        byte[] bin = row.getBinary(ord);
+        if (bin == null) {
+          System.out.print("null");
+        } else {
+          System.out.print("[");
+          for (int i = 0; i < bin.length; ++i) {
+            if (i != 0) {
+              System.out.print(", ");
+            }
+            int v = bin[i] & 0xff;
+            if (v < 16) {
+              System.out.print("0" + Integer.toHexString(v));
+            } else {
+              System.out.print(Integer.toHexString(v));
+            }
+          }
+          System.out.print("]");
+        }
+        break;
+      }
+      case DECIMAL:
+        System.out.print(row.getDecimal(ord, schema.getPrecision(), 
schema.getScale()));
+        break;
+      case DATE:
+        System.out.print("\"" + new DateWritable(row.getInt(ord)) + "\"");
+        break;
+      case TIMESTAMP:
+        System.out.print("\"" + new Timestamp(row.getLong(ord)) + "\"");
+        break;
+      case STRUCT:
+        printRow(row.getStruct(ord, schema.getChildren().size()), schema);
+        break;
+      case LIST: {
+        TypeDescription child = schema.getChildren().get(0);
+        System.out.print("[");
+        ArrayData list = row.getArray(ord);
+        for(int e=0; e < list.numElements(); ++e) {
+          if (e != 0) {
+            System.out.print(", ");
+          }
+          printRow(list, e, child);
+        }
+        System.out.print("]");
+        break;
+      }
+      case MAP: {
+        TypeDescription keyType = schema.getChildren().get(0);
+        TypeDescription valueType = schema.getChildren().get(1);
+        MapData map = row.getMap(ord);
+        ArrayData keys = map.keyArray();
+        ArrayData values = map.valueArray();
+        System.out.print("[");
+        for(int e=0; e < map.numElements(); ++e) {
+          if (e != 0) {
+            System.out.print(", ");
+          }
+          printRow(keys, e, keyType);
+          System.out.print(": ");
+          printRow(values, e, valueType);
+        }
+        System.out.print("]");
+        break;
+      }
+      default:
+        throw new IllegalArgumentException("Unhandled type " + schema);
+    }
+  }
+
+  private static int getArrayElementSize(TypeDescription type) {
+    switch (type.getCategory()) {
+      case BOOLEAN:
+      case BYTE:
+        return 1;
+      case SHORT:
+        return 2;
+      case INT:
+      case FLOAT:
+        return 4;
+      default:
+        return 8;
+    }
+  }
+
+  /**
+   * The common interface for converting from a ORC ColumnVector to a Spark
+   * UnsafeRow. UnsafeRows need two different interfaces for writers and thus
+   * we have two methods the first is for structs (UnsafeRowWriter) and the
+   * second is for lists and maps (UnsafeArrayWriter). If Spark adds a common
+   * interface similar to SpecializedGetters we could that and a single set of
+   * methods.
+   */
+  interface Converter {
+    void convert(UnsafeRowWriter writer, int column, ColumnVector vector, int 
row);
+    void convert(UnsafeArrayWriter writer, int element, ColumnVector vector, 
int row);
+  }
+
+  private static class BooleanConverter implements Converter {
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, ((LongColumnVector) vector).vector[row] != 0);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, ((LongColumnVector) vector).vector[row] != 0);
+      }
+    }
+  }
+
+  private static class ByteConverter implements Converter {
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector,
+                        int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, (byte) ((LongColumnVector) vector).vector[row]);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, (byte) ((LongColumnVector) vector).vector[row]);
+      }
+    }
+  }
+
+  private static class ShortConverter implements Converter {
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector,
+                        int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, (short) ((LongColumnVector) vector).vector[row]);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, (short) ((LongColumnVector) vector).vector[row]);
+      }
+    }
+  }
+
+  private static class IntConverter implements Converter {
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector,
+                        int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, (int) ((LongColumnVector) vector).vector[row]);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, (int) ((LongColumnVector) vector).vector[row]);
+      }
+    }
+  }
+
+  private static class LongConverter implements Converter {
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector,
+                        int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, ((LongColumnVector) vector).vector[row]);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, ((LongColumnVector) vector).vector[row]);
+      }
+    }
+  }
+
+  private static class FloatConverter implements Converter {
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector,
+                        int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, (float) ((DoubleColumnVector) 
vector).vector[row]);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, (float) ((DoubleColumnVector) 
vector).vector[row]);
+      }
+    }
+  }
+
+  private static class DoubleConverter implements Converter {
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector,
+                        int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, ((DoubleColumnVector) vector).vector[row]);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, ((DoubleColumnVector) vector).vector[row]);
+      }
+    }
+  }
+
+  private static class TimestampConverter implements Converter {
+
+    private long convert(TimestampColumnVector vector, int row) {
+      // compute microseconds past 1970.
+      long micros = (vector.time[row]/1000) * 1_000_000 + vector.nanos[row] / 
1000;
+      return micros;
+    }
+
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector,
+                        int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        writer.write(column, convert((TimestampColumnVector) vector, row));
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element,
+                        ColumnVector vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        writer.write(element, convert((TimestampColumnVector) vector, row));
+      }
+    }
+  }
+
+  private static class BinaryConverter implements Converter {
+
+    @Override
+    public void convert(UnsafeRowWriter writer, int column, ColumnVector 
vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNullAt(column);
+      } else {
+        BytesColumnVector v = (BytesColumnVector) vector;
+        writer.write(column, v.vector[row], v.start[row], v.length[row]);
+      }
+    }
+
+    @Override
+    public void convert(UnsafeArrayWriter writer, int element, ColumnVector 
vector, int row) {
+      if (vector.isRepeating) {
+        row = 0;
+      }
+      if (!vector.noNulls && vector.isNull[row]) {
+        writer.setNull(element);
+      } else {
+        final BytesColumnVector v = (BytesColumnVector) vector;
+        writer.write(element, v.vector[row], v.start[row], v.length[row]);
+      }
+    }
+  }
+
+  /**
+   * This hack is to get the unscaled value (for precision <= 18) quickly.
+   * This can be replaced when we upgrade to storage-api 2.5.0.
 
 Review comment:
   Is storage-api 2.5.0 available?

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to