From cf5b191be2135ceaae8fc71d14a0e8c1fffc4867 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Mon, 18 Mar 2019 14:54:56 -0700 Subject: [PATCH 01/23] Add missing components for ORC support in spark --- build.gradle | 3 + .../iceberg/spark/data/SparkOrcReader.java | 870 ++++++++++++++++++ .../iceberg/spark/data/SparkOrcWriter.java | 434 +++++++++ .../iceberg/spark/source/TestOrcScan.java | 137 +++ 4 files changed, 1444 insertions(+) create mode 100644 spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java create mode 100644 spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcWriter.java create mode 100644 spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcScan.java diff --git a/build.gradle b/build.gradle index ae8920ee2961..32f2e07355a9 100644 --- a/build.gradle +++ b/build.gradle @@ -264,6 +264,7 @@ project(':iceberg-spark') { compile project(':iceberg-api') compile project(':iceberg-common') compile project(':iceberg-core') + compile project(':iceberg-orc') compile project(':iceberg-parquet') compileOnly "org.apache.avro:avro:$avroVersion" @@ -367,11 +368,13 @@ project(':iceberg-presto-runtime') { dependencies { shadow project(':iceberg-api') shadow project(':iceberg-core') + shadow project(':iceberg-orc') shadow project(':iceberg-parquet') shadow project(':iceberg-hive') shadow "org.apache.parquet:parquet-avro:$parquetVersion" shadow "org.apache.avro:avro:$avroVersion" + shadow "org.apache.orc:orc-core:$orcVersion:nohive" shadow ("org.apache.hive:hive-metastore:$hiveVersion") { exclude group: 'org.apache.hadoop', module: 'hadoop-common' // exclude group: 'org.apache.orc', module: 'orc-core' diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java new file mode 100644 index 000000000000..6be855ece02d --- /dev/null +++ b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java @@ -0,0 +1,870 @@ +/* + * 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 com.netflix.iceberg.spark.data; + +import com.netflix.iceberg.FileScanTask; +import com.netflix.iceberg.Schema; +import com.netflix.iceberg.io.InputFile; +import com.netflix.iceberg.orc.ColumnIdMap; +import com.netflix.iceberg.orc.ORC; +import com.netflix.iceberg.orc.OrcIterator; +import com.netflix.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.BufferHolder; +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter; +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; +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 org.apache.spark.unsafe.array.ByteArrayMethods; + +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, Closeable { + private final static int INITIAL_SIZE = 128 * 1024; + private final OrcIterator reader; + private final TypeDescription orcSchema; + private final UnsafeRow row; + private final BufferHolder holder; + 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(); + orcSchema = TypeConversion.toOrc(readSchema, columnIds); + reader = ORC.read(location) + .split(task.start(), task.length()) + .schema(readSchema) + .build(); + int numFields = readSchema.columns().size(); + row = new UnsafeRow(numFields); + holder = new BufferHolder(row, INITIAL_SIZE); + writer = new UnsafeRowWriter(holder, numFields); + converter = new Converter[numFields]; + for(int c=0; c < numFields; ++c) { + converter[c] = buildConverter(holder, 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; + } + // Reset the holder to start the buffer over again. + // BufferHolder.reset does the wrong thing... + holder.cursor = Platform.BYTE_ARRAY_OFFSET; + writer.reset(); + for(int c=0; c < current.cols.length; ++c) { + converter[c].convert(writer, c, current.cols[c], nextRow); + } + nextRow++; + return row; + } + + @Override + public void close() throws IOException { + reader.close(); + } + + private static void printRow(SpecializedGetters row, TypeDescription schema) { + List 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); + } + } + 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)); + } + } + } + + /** + * UnsafeArrayWriter doesn't have a binary form that lets the user pass an + * offset and length, so I've added one here. It is the minor tweak of the + * UnsafeArrayWriter.write(int, byte[]) method. + * @param holder the BufferHolder where the bytes are being written + * @param writer the UnsafeArrayWriter + * @param ordinal the element that we are writing into + * @param input the input bytes + * @param offset the first byte from input + * @param length the number of bytes to write + */ + static void write(BufferHolder holder, UnsafeArrayWriter writer, int ordinal, + byte[] input, int offset, int length) { + final int roundedSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(length); + + // grow the global buffer before writing data. + holder.grow(roundedSize); + + if ((length & 0x07) > 0) { + Platform.putLong(holder.buffer, holder.cursor + ((length >> 3) << 3), 0L); + } + + // Write the bytes to the variable length portion. + Platform.copyMemory(input, Platform.BYTE_ARRAY_OFFSET + offset, + holder.buffer, holder.cursor, length); + + writer.setOffsetAndSize(ordinal, holder.cursor, length); + + // move the cursor forward. + holder.cursor += roundedSize; + } + + private static class BinaryConverter implements Converter { + private final BufferHolder holder; + + BinaryConverter(BufferHolder holder) { + this.holder = holder; + } + + @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 { + BytesColumnVector v = (BytesColumnVector) vector; + write(holder, writer, 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. + */ + static class DecimalHack extends FastHiveDecimal { + long unscaledLong(FastHiveDecimal value) { + fastSet(value); + return fastSignum * fast1 * 10_000_000_000_000_000L + fast0; + } + } + + private static class Decimal18Converter implements Converter { + final DecimalHack hack = new DecimalHack(); + final int precision; + final int scale; + + Decimal18Converter(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @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 { + HiveDecimalWritable v = ((DecimalColumnVector) vector).vector[row]; + writer.write(column, + new Decimal().set(hack.unscaledLong(v), precision, v.scale()), + precision, scale); + } + } + + @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 { + HiveDecimalWritable v = ((DecimalColumnVector) vector).vector[row]; + writer.write(element, + new Decimal().set(hack.unscaledLong(v), precision, v.scale()), + precision, scale); + } + } + } + + private static class Decimal38Converter implements Converter { + final int precision; + final int scale; + + Decimal38Converter(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @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 { + BigDecimal v = ((DecimalColumnVector) vector).vector[row] + .getHiveDecimal().bigDecimalValue(); + writer.write(column, + new Decimal().set(new scala.math.BigDecimal(v), precision, scale), + precision, scale); + } + } + + @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 { + BigDecimal v = ((DecimalColumnVector) vector).vector[row] + .getHiveDecimal().bigDecimalValue(); + writer.write(element, + new Decimal().set(new scala.math.BigDecimal(v), precision, scale), + precision, scale); + } + } + } + + private static class StructConverter implements Converter { + private final BufferHolder holder; + private final Converter[] children; + private final UnsafeRowWriter childWriter; + + StructConverter(BufferHolder holder, TypeDescription schema) { + this.holder = holder; + children = new Converter[schema.getChildren().size()]; + for(int c=0; c < children.length; ++c) { + children[c] = buildConverter(holder, schema.getChildren().get(c)); + } + childWriter = new UnsafeRowWriter(holder, children.length); + } + + int writeStruct(StructColumnVector vector, int row) { + int start = holder.cursor; + childWriter.reset(); + for(int c=0; c < children.length; ++c) { + children[c].convert(childWriter, c, vector.fields[c], row); + } + return start; + } + + @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 { + int start = writeStruct((StructColumnVector) vector, row); + writer.setOffsetAndSize(column, start, holder.cursor - start); + } + } + + @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 { + int start = writeStruct((StructColumnVector) vector, row); + writer.setOffsetAndSize(element, start, holder.cursor - start); + } + } + } + + private static class ListConverter implements Converter { + private final BufferHolder holder; + private final Converter children; + private final UnsafeArrayWriter childWriter; + private final int elementSize; + + ListConverter(BufferHolder holder, TypeDescription schema) { + this.holder = holder; + TypeDescription child = schema.getChildren().get(0); + children = buildConverter(holder, child); + childWriter = new UnsafeArrayWriter(); + elementSize = getArrayElementSize(child); + } + + int writeList(ListColumnVector v, int row) { + int offset = (int) v.offsets[row]; + int length = (int) v.lengths[row]; + int start = holder.cursor; + childWriter.initialize(holder, length, elementSize); + for(int c=0; c < length; ++c) { + children.convert(childWriter, c, v.child, offset + c); + } + return start; + } + + @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 { + int start = writeList((ListColumnVector) vector, row); + writer.setOffsetAndSize(column, start, holder.cursor - start); + } + } + + @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 { + int start = writeList((ListColumnVector) vector, row); + writer.setOffsetAndSize(element, start, holder.cursor - start); + } + } + } + + private static class MapConverter implements Converter { + private final BufferHolder holder; + private final Converter keyConvert; + private final Converter valueConvert; + private final UnsafeArrayWriter childWriter; + private final int keySize; + private final int valueSize; + + MapConverter(BufferHolder holder, TypeDescription schema) { + this.holder = holder; + TypeDescription keyType = schema.getChildren().get(0); + TypeDescription valueType = schema.getChildren().get(1); + keyConvert = buildConverter(holder, keyType); + keySize = getArrayElementSize(keyType); + valueConvert = buildConverter(holder, valueType); + valueSize = getArrayElementSize(valueType); + childWriter = new UnsafeArrayWriter(); + } + + int writeMap(MapColumnVector v, int row) { + int offset = (int) v.offsets[row]; + int length = (int) v.lengths[row]; + int start = holder.cursor; + // save room for the key size + final int KEY_SIZE_BYTES = 8; + holder.grow(KEY_SIZE_BYTES); + holder.cursor += KEY_SIZE_BYTES; + // serialize the keys + childWriter.initialize(holder, length, keySize); + for(int c=0; c < length; ++c) { + keyConvert.convert(childWriter, c, v.keys, offset + c); + } + // store the serialized size of the keys + Platform.putLong(holder.buffer, start, holder.cursor - start - KEY_SIZE_BYTES); + // serialize the values + childWriter.initialize(holder, length, valueSize); + for(int c=0; c < length; ++c) { + valueConvert.convert(childWriter, c, v.values, offset + c); + } + return start; + } + + @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 { + int start = writeMap((MapColumnVector) vector, row); + writer.setOffsetAndSize(column, start, holder.cursor - start); + } + } + + @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 { + int start = writeMap((MapColumnVector) vector, row); + writer.setOffsetAndSize(element, start, holder.cursor - start); + } + } + } + + static Converter buildConverter(BufferHolder holder, TypeDescription schema) { + switch (schema.getCategory()) { + case BOOLEAN: + return new BooleanConverter(); + case BYTE: + return new ByteConverter(); + case SHORT: + return new ShortConverter(); + case DATE: + case INT: + return new IntConverter(); + case LONG: + return new LongConverter(); + case FLOAT: + return new FloatConverter(); + case DOUBLE: + return new DoubleConverter(); + case TIMESTAMP: + return new TimestampConverter(); + case DECIMAL: + if (schema.getPrecision() <= Decimal.MAX_LONG_DIGITS()) { + return new Decimal18Converter(schema.getPrecision(), schema.getScale()); + } else { + return new Decimal38Converter(schema.getPrecision(), schema.getScale()); + } + case BINARY: + case STRING: + case CHAR: + case VARCHAR: + return new BinaryConverter(holder); + case STRUCT: + return new StructConverter(holder, schema); + case LIST: + return new ListConverter(holder, schema); + case MAP: + return new MapConverter(holder, schema); + default: + throw new IllegalArgumentException("Unhandled type " + schema); + } + } +} diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcWriter.java b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcWriter.java new file mode 100644 index 000000000000..175be10483ce --- /dev/null +++ b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcWriter.java @@ -0,0 +1,434 @@ +/* + * 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 com.netflix.iceberg.spark.data; + +import com.netflix.iceberg.Metrics; +import com.netflix.iceberg.io.FileAppender; +import com.netflix.iceberg.orc.OrcFileAppender; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.common.type.HiveDecimal; +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.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; +import java.sql.Timestamp; +import java.util.List; + +/** + * This class acts as an adaptor from an OrcFileAppender to a + * FileAppender<InternalRow>. + */ +public class SparkOrcWriter implements FileAppender { + private final static int BATCH_SIZE = 1024; + private final VectorizedRowBatch batch; + private final OrcFileAppender writer; + private final Converter[] converters; + + public SparkOrcWriter(OrcFileAppender writer) { + TypeDescription schema = writer.getSchema(); + batch = schema.createRowBatch(BATCH_SIZE); + this.writer = writer; + converters = buildConverters(schema); + } + + /** + * The interface for the conversion from Spark's SpecializedGetters to + * ORC's ColumnVectors. + */ + interface Converter { + /** + * Take a value from the Spark data value and add it to the ORC output. + * @param rowId the row in the ColumnVector + * @param column either the column number or element number + * @param data either an InternalRow or ArrayData + * @param output the ColumnVector to put the value into + */ + void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output); + } + + static class BooleanConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((LongColumnVector) output).vector[rowId] = data.getBoolean(column) ? 1 : 0; + } + } + } + + static class ByteConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((LongColumnVector) output).vector[rowId] = data.getByte(column); + } + } + } + + static class ShortConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((LongColumnVector) output).vector[rowId] = data.getShort(column); + } + } + } + + static class IntConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((LongColumnVector) output).vector[rowId] = data.getInt(column); + } + } + } + + static class LongConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((LongColumnVector) output).vector[rowId] = data.getLong(column); + } + } + } + + static class FloatConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((DoubleColumnVector) output).vector[rowId] = data.getFloat(column); + } + } + } + + static class DoubleConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((DoubleColumnVector) output).vector[rowId] = data.getDouble(column); + } + } + } + + static class StringConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + byte[] value = data.getUTF8String(column).getBytes(); + ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); + } + } + } + + static class BytesConverter implements Converter { + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + // getBinary always makes a copy, so we don't need to worry about it + // being changed behind our back. + byte[] value = data.getBinary(column); + ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); + } + } + } + + static class TimestampConverter implements Converter { + + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + TimestampColumnVector cv = (TimestampColumnVector) output; + long micros = data.getLong(column); + cv.time[rowId] = (micros / 1_000_000) * 1000; + int nanos = (int) (micros % 1_000_000) * 1000; + if (nanos < 0) { + nanos += 1_000_000_000; + cv.time[rowId] -= 1000; + } + cv.nanos[rowId] = nanos; + } + } + } + + static class Decimal18Converter implements Converter { + private final int precision; + private final int scale; + + Decimal18Converter(TypeDescription schema) { + precision = schema.getPrecision(); + scale = schema.getScale(); + } + + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((DecimalColumnVector) output).vector[rowId].setFromLongAndScale( + data.getDecimal(column, precision, scale).toUnscaledLong(), scale); + } + } + } + + static class Decimal38Converter implements Converter { + private final int precision; + private final int scale; + + Decimal38Converter(TypeDescription schema) { + precision = schema.getPrecision(); + scale = schema.getScale(); + } + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ((DecimalColumnVector) output).vector[rowId].set( + HiveDecimal.create(data.getDecimal(column, precision, scale) + .toJavaBigDecimal())); + } + } + } + + static class StructConverter implements Converter { + private final Converter[] children; + + StructConverter(TypeDescription schema) { + children = new Converter[schema.getChildren().size()]; + for(int c=0; c < children.length; ++c) { + children[c] = buildConverter(schema.getChildren().get(c)); + } + } + + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + InternalRow value = data.getStruct(column, children.length); + StructColumnVector cv = (StructColumnVector) output; + for(int c=0; c < children.length; ++c) { + children[c].addValue(rowId, c, value, cv.fields[c]); + } + } + } + } + + static class ListConverter implements Converter { + private final Converter children; + + ListConverter(TypeDescription schema) { + children = buildConverter(schema.getChildren().get(0)); + } + + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + ArrayData value = data.getArray(column); + ListColumnVector cv = (ListColumnVector) output; + // record the length and start of the list elements + cv.lengths[rowId] = value.numElements(); + cv.offsets[rowId] = cv.childCount; + cv.childCount += cv.lengths[rowId]; + // make sure the child is big enough + cv.child.ensureSize(cv.childCount, true); + // Add each element + for(int e=0; e < cv.lengths[rowId]; ++e) { + children.addValue((int) (e + cv.offsets[rowId]), e, value, cv.child); + } + } + } + } + + static class MapConverter implements Converter { + private final Converter keyConverter; + private final Converter valueConverter; + + MapConverter(TypeDescription schema) { + keyConverter = buildConverter(schema.getChildren().get(0)); + valueConverter = buildConverter(schema.getChildren().get(1)); + } + + public void addValue(int rowId, int column, SpecializedGetters data, + ColumnVector output) { + if (data.isNullAt(column)) { + output.noNulls = false; + output.isNull[rowId] = true; + } else { + output.isNull[rowId] = false; + MapData map = data.getMap(column); + ArrayData key = map.keyArray(); + ArrayData value = map.valueArray(); + MapColumnVector cv = (MapColumnVector) output; + // record the length and start of the list elements + cv.lengths[rowId] = value.numElements(); + cv.offsets[rowId] = cv.childCount; + cv.childCount += cv.lengths[rowId]; + // make sure the child is big enough + cv.keys.ensureSize(cv.childCount, true); + cv.values.ensureSize(cv.childCount, true); + // Add each element + for(int e=0; e < cv.lengths[rowId]; ++e) { + int pos = (int)(e + cv.offsets[rowId]); + keyConverter.addValue(pos, e, key, cv.keys); + valueConverter.addValue(pos, e, value, cv.values); + } + } + } + } + + private static Converter buildConverter(TypeDescription schema) { + switch (schema.getCategory()) { + case BOOLEAN: + return new BooleanConverter(); + case BYTE: + return new ByteConverter(); + case SHORT: + return new ShortConverter(); + case DATE: + case INT: + return new IntConverter(); + case LONG: + return new LongConverter(); + case FLOAT: + return new FloatConverter(); + case DOUBLE: + return new DoubleConverter(); + case BINARY: + return new BytesConverter(); + case STRING: + case CHAR: + case VARCHAR: + return new StringConverter(); + case DECIMAL: + return schema.getPrecision() <= 18 + ? new Decimal18Converter(schema) + : new Decimal38Converter(schema); + case TIMESTAMP: + return new TimestampConverter(); + case STRUCT: + return new StructConverter(schema); + case LIST: + return new ListConverter(schema); + case MAP: + return new MapConverter(schema); + } + throw new IllegalArgumentException("Unhandled type " + schema); + } + + private static Converter[] buildConverters(TypeDescription schema) { + if (schema.getCategory() != TypeDescription.Category.STRUCT) { + throw new IllegalArgumentException("Top level must be a struct " + schema); + } + List children = schema.getChildren(); + Converter[] result = new Converter[children.size()]; + for(int c=0; c < children.size(); ++c) { + result[c] = buildConverter(children.get(c)); + } + return result; + } + + @Override + public void add(InternalRow datum) { + int row = batch.size++; + for(int c=0; c < converters.length; ++c) { + converters[c].addValue(row, c, datum, batch.cols[c]); + } + if (batch.size == BATCH_SIZE) { + writer.add(batch); + batch.reset(); + } + } + + @Override + public Metrics metrics() { + return writer.metrics(); + } + + @Override + public void close() throws IOException { + if (batch.size > 0) { + writer.add(batch); + batch.reset(); + } + writer.close(); + } +} diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcScan.java b/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcScan.java new file mode 100644 index 000000000000..4a6fb269b441 --- /dev/null +++ b/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcScan.java @@ -0,0 +1,137 @@ +/* + * 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 com.netflix.iceberg.spark.source; + +import com.netflix.iceberg.DataFile; +import com.netflix.iceberg.DataFiles; +import com.netflix.iceberg.FileFormat; +import com.netflix.iceberg.Metrics; +import com.netflix.iceberg.PartitionSpec; +import com.netflix.iceberg.Schema; +import com.netflix.iceberg.Table; +import com.netflix.iceberg.hadoop.HadoopTables; +import com.netflix.iceberg.io.FileAppender; +import com.netflix.iceberg.orc.ORC; +import com.netflix.iceberg.orc.OrcFileAppender; +import com.netflix.iceberg.spark.data.AvroDataTest; +import com.netflix.iceberg.spark.data.RandomData; +import com.netflix.iceberg.spark.data.SparkOrcWriter; +import com.netflix.iceberg.spark.data.TestHelpers; +import com.netflix.iceberg.types.Type; +import com.netflix.iceberg.types.Types; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.orc.storage.serde2.io.DateWritable; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; + +import static com.netflix.iceberg.Files.localOutput; + +public class TestOrcScan extends AvroDataTest { + private static final Configuration CONF = new Configuration(); + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private static SparkSession spark = null; + + @BeforeClass + public static void startSpark() { + TestOrcScan.spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @AfterClass + public static void stopSpark() { + SparkSession spark = TestOrcScan.spark; + TestOrcScan.spark = null; + spark.stop(); + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + System.out.println("Starting ORC test with " + schema); + final int ROW_COUNT = 100; + final long SEED = 1; + File parent = temp.newFolder("orc"); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + dataFolder.mkdirs(); + + File orcFile = new File(dataFolder, + FileFormat.ORC.addExtension(UUID.randomUUID().toString())); + + HadoopTables tables = new HadoopTables(CONF); + Table table = tables.create(schema, PartitionSpec.unpartitioned(), + location.toString()); + + // Important: use the table's schema for the rest of the test + // When tables are created, the column ids are reassigned. + Schema tableSchema = table.schema(); + + Metrics metrics; + SparkOrcWriter writer = new SparkOrcWriter(ORC.write(localOutput(orcFile)) + .schema(tableSchema) + .build()); + try { + writer.addAll(RandomData.generateSpark(tableSchema, ROW_COUNT, SEED)); + } finally { + writer.close(); + // close writes the last batch, so metrics are not correct until after close is called + metrics = writer.metrics(); + } + + DataFile file = DataFiles.builder(PartitionSpec.unpartitioned()) + .withFileSizeInBytes(orcFile.length()) + .withPath(orcFile.toString()) + .withMetrics(metrics) + .build(); + + table.newAppend().appendFile(file).commit(); + + Dataset df = spark.read() + .format("iceberg") + .load(location.toString()); + + List rows = df.collectAsList(); + Assert.assertEquals("Wrong number of rows", ROW_COUNT, rows.size()); + Iterator expected = RandomData.generateSpark(tableSchema, + ROW_COUNT, SEED); + for(int i=0; i < ROW_COUNT; ++i) { + TestHelpers.assertEquals("row " + i, schema.asStruct(), expected.next(), + rows.get(i)); + } + } +} From e39fd43866f64661917c53d5edb660d967c3720f Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Mon, 18 Mar 2019 17:44:34 -0700 Subject: [PATCH 02/23] Add ORC support for spark 2.4.0 --- .../iceberg/spark/data/SparkOrcReader.java | 191 +++++++----------- .../iceberg/spark/data/SparkOrcWriter.java | 8 +- .../apache/iceberg/spark/source/Reader.java | 30 ++- .../iceberg/spark/source/TestOrcScan.java | 40 ++-- 4 files changed, 122 insertions(+), 147 deletions(-) rename spark/src/main/java/{com/netflix => org/apache}/iceberg/spark/data/SparkOrcWriter.java (98%) rename spark/src/test/java/{com/netflix => org/apache}/iceberg/spark/source/TestOrcScan.java (82%) diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java index 6be855ece02d..ce7ce8f4dcf4 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java @@ -14,15 +14,15 @@ * limitations under the License. */ -package com.netflix.iceberg.spark.data; - -import com.netflix.iceberg.FileScanTask; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.orc.ColumnIdMap; -import com.netflix.iceberg.orc.ORC; -import com.netflix.iceberg.orc.OrcIterator; -import com.netflix.iceberg.orc.TypeConversion; +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; @@ -40,14 +40,13 @@ 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.BufferHolder; 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 org.apache.spark.unsafe.array.ByteArrayMethods; import java.io.Closeable; import java.io.IOException; @@ -66,8 +65,6 @@ public class SparkOrcReader implements Iterator, Closeable { private final static int INITIAL_SIZE = 128 * 1024; private final OrcIterator reader; private final TypeDescription orcSchema; - private final UnsafeRow row; - private final BufferHolder holder; private final UnsafeRowWriter writer; private int nextRow = 0; private VectorizedRowBatch current = null; @@ -83,12 +80,10 @@ public SparkOrcReader(InputFile location, .schema(readSchema) .build(); int numFields = readSchema.columns().size(); - row = new UnsafeRow(numFields); - holder = new BufferHolder(row, INITIAL_SIZE); - writer = new UnsafeRowWriter(holder, numFields); + writer = new UnsafeRowWriter(numFields, INITIAL_SIZE); converter = new Converter[numFields]; - for(int c=0; c < numFields; ++c) { - converter[c] = buildConverter(holder, orcSchema.getChildren().get(c)); + for(int c = 0; c < numFields; ++c) { + converter[c] = buildConverter(writer, orcSchema.getChildren().get(c)); } } @@ -105,13 +100,12 @@ public UnsafeRow next() { } // Reset the holder to start the buffer over again. // BufferHolder.reset does the wrong thing... - holder.cursor = Platform.BYTE_ARRAY_OFFSET; writer.reset(); for(int c=0; c < current.cols.length; ++c) { converter[c].convert(writer, c, current.cols[c], nextRow); } nextRow++; - return row; + return writer.getRow(); } @Override @@ -250,8 +244,7 @@ static int getArrayElementSize(TypeDescription type) { */ interface Converter { void convert(UnsafeRowWriter writer, int column, ColumnVector vector, int row); - void convert(UnsafeArrayWriter writer, int element, ColumnVector vector, - int row); + void convert(UnsafeArrayWriter writer, int element, ColumnVector vector, int row); } private static class BooleanConverter implements Converter { @@ -484,48 +477,10 @@ public void convert(UnsafeArrayWriter writer, int element, } } - /** - * UnsafeArrayWriter doesn't have a binary form that lets the user pass an - * offset and length, so I've added one here. It is the minor tweak of the - * UnsafeArrayWriter.write(int, byte[]) method. - * @param holder the BufferHolder where the bytes are being written - * @param writer the UnsafeArrayWriter - * @param ordinal the element that we are writing into - * @param input the input bytes - * @param offset the first byte from input - * @param length the number of bytes to write - */ - static void write(BufferHolder holder, UnsafeArrayWriter writer, int ordinal, - byte[] input, int offset, int length) { - final int roundedSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(length); - - // grow the global buffer before writing data. - holder.grow(roundedSize); - - if ((length & 0x07) > 0) { - Platform.putLong(holder.buffer, holder.cursor + ((length >> 3) << 3), 0L); - } - - // Write the bytes to the variable length portion. - Platform.copyMemory(input, Platform.BYTE_ARRAY_OFFSET + offset, - holder.buffer, holder.cursor, length); - - writer.setOffsetAndSize(ordinal, holder.cursor, length); - - // move the cursor forward. - holder.cursor += roundedSize; - } - private static class BinaryConverter implements Converter { - private final BufferHolder holder; - - BinaryConverter(BufferHolder holder) { - this.holder = holder; - } @Override - public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, - int row) { + public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, int row) { if (vector.isRepeating) { row = 0; } @@ -538,17 +493,15 @@ public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, } @Override - public void convert(UnsafeArrayWriter writer, int element, - ColumnVector vector, int row) { + 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 { - BytesColumnVector v = (BytesColumnVector) vector; - write(holder, writer, element, v.vector[row], v.start[row], - v.length[row]); + final BytesColumnVector v = (BytesColumnVector) vector; + writer.write(element, v.vector[row], v.start[row], v.length[row]); } } } @@ -652,21 +605,19 @@ public void convert(UnsafeArrayWriter writer, int element, } private static class StructConverter implements Converter { - private final BufferHolder holder; private final Converter[] children; private final UnsafeRowWriter childWriter; - StructConverter(BufferHolder holder, TypeDescription schema) { - this.holder = holder; + StructConverter(final UnsafeWriter parentWriter, final TypeDescription schema) { children = new Converter[schema.getChildren().size()]; for(int c=0; c < children.length; ++c) { - children[c] = buildConverter(holder, schema.getChildren().get(c)); + children[c] = buildConverter(parentWriter, schema.getChildren().get(c)); } - childWriter = new UnsafeRowWriter(holder, children.length); + childWriter = new UnsafeRowWriter(parentWriter, children.length); } int writeStruct(StructColumnVector vector, int row) { - int start = holder.cursor; + int start = childWriter.cursor(); childWriter.reset(); for(int c=0; c < children.length; ++c) { children[c].convert(childWriter, c, vector.fields[c], row); @@ -683,7 +634,7 @@ public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, int writer.setNullAt(column); } else { int start = writeStruct((StructColumnVector) vector, row); - writer.setOffsetAndSize(column, start, holder.cursor - start); + writer.setOffsetAndSizeFromPreviousCursor(column, start); } } @@ -697,35 +648,32 @@ public void convert(UnsafeArrayWriter writer, int element, writer.setNull(element); } else { int start = writeStruct((StructColumnVector) vector, row); - writer.setOffsetAndSize(element, start, holder.cursor - start); + writer.setOffsetAndSizeFromPreviousCursor(element, start); } } } private static class ListConverter implements Converter { - private final BufferHolder holder; private final Converter children; private final UnsafeArrayWriter childWriter; - private final int elementSize; - ListConverter(BufferHolder holder, TypeDescription schema) { - this.holder = holder; + ListConverter(final UnsafeWriter parentWriter, TypeDescription schema) { TypeDescription child = schema.getChildren().get(0); - children = buildConverter(holder, child); - childWriter = new UnsafeArrayWriter(); - elementSize = getArrayElementSize(child); + children = buildConverter(parentWriter, child); + childWriter = new UnsafeArrayWriter(parentWriter, getArrayElementSize(child)); + } int writeList(ListColumnVector v, int row) { int offset = (int) v.offsets[row]; int length = (int) v.lengths[row]; - int start = holder.cursor; - childWriter.initialize(holder, length, elementSize); + int start = childWriter.cursor(); + childWriter.initialize(length); for(int c=0; c < length; ++c) { children.convert(childWriter, c, v.child, offset + c); } return start; - } + } @Override public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, @@ -737,7 +685,7 @@ public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, writer.setNullAt(column); } else { int start = writeList((ListColumnVector) vector, row); - writer.setOffsetAndSize(column, start, holder.cursor - start); + writer.setOffsetAndSizeFromPreviousCursor(column, start); } } @@ -751,56 +699,62 @@ public void convert(UnsafeArrayWriter writer, int element, writer.setNull(element); } else { int start = writeList((ListColumnVector) vector, row); - writer.setOffsetAndSize(element, start, holder.cursor - start); + writer.setOffsetAndSizeFromPreviousCursor(element, start); } } } private static class MapConverter implements Converter { - private final BufferHolder holder; private final Converter keyConvert; private final Converter valueConvert; - private final UnsafeArrayWriter childWriter; + + private final UnsafeArrayWriter keyWriter; + private final UnsafeArrayWriter valueWriter; + private final int keySize; private final int valueSize; - MapConverter(BufferHolder holder, TypeDescription schema) { - this.holder = holder; - TypeDescription keyType = schema.getChildren().get(0); - TypeDescription valueType = schema.getChildren().get(1); - keyConvert = buildConverter(holder, keyType); + private final int KEY_SIZE_BYTES = 8; + + MapConverter(final UnsafeWriter parentWriter, TypeDescription schema) { + final TypeDescription keyType = schema.getChildren().get(0); + final TypeDescription valueType = schema.getChildren().get(1); + keyConvert = buildConverter(parentWriter, keyType); keySize = getArrayElementSize(keyType); - valueConvert = buildConverter(holder, valueType); + keyWriter = new UnsafeArrayWriter(parentWriter, keySize); + valueConvert = buildConverter(parentWriter, valueType); valueSize = getArrayElementSize(valueType); - childWriter = new UnsafeArrayWriter(); + valueWriter = new UnsafeArrayWriter(parentWriter, valueSize); } int writeMap(MapColumnVector v, int row) { - int offset = (int) v.offsets[row]; - int length = (int) v.lengths[row]; - int start = holder.cursor; + final int offset = (int) v.offsets[row]; + final int length = (int) v.lengths[row]; + final int start = keyWriter.cursor(); + // save room for the key size - final int KEY_SIZE_BYTES = 8; - holder.grow(KEY_SIZE_BYTES); - holder.cursor += KEY_SIZE_BYTES; + keyWriter.grow(KEY_SIZE_BYTES); + keyWriter.increaseCursor(KEY_SIZE_BYTES); + // serialize the keys - childWriter.initialize(holder, length, keySize); - for(int c=0; c < length; ++c) { - keyConvert.convert(childWriter, c, v.keys, offset + c); + keyWriter.initialize(length); + for(int c = 0; c < length; ++c) { + keyConvert.convert(keyWriter, c, v.keys, offset + c); } // store the serialized size of the keys - Platform.putLong(holder.buffer, start, holder.cursor - start - KEY_SIZE_BYTES); + Platform.putLong(keyWriter.getBuffer(), start, + keyWriter.cursor() - start - KEY_SIZE_BYTES); + // serialize the values - childWriter.initialize(holder, length, valueSize); - for(int c=0; c < length; ++c) { - valueConvert.convert(childWriter, c, v.values, offset + c); + valueWriter.initialize(length); + for(int c = 0; c < length; ++c) { + valueConvert.convert(valueWriter, c, v.values, offset + c); } return start; } @Override - public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, - int row) { + public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, int row) { if (vector.isRepeating) { row = 0; } @@ -808,13 +762,12 @@ public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, writer.setNullAt(column); } else { int start = writeMap((MapColumnVector) vector, row); - writer.setOffsetAndSize(column, start, holder.cursor - start); + writer.setOffsetAndSizeFromPreviousCursor(column, start); } } @Override - public void convert(UnsafeArrayWriter writer, int element, - ColumnVector vector, int row) { + public void convert(UnsafeArrayWriter writer, int element, ColumnVector vector, int row) { if (vector.isRepeating) { row = 0; } @@ -822,12 +775,12 @@ public void convert(UnsafeArrayWriter writer, int element, writer.setNull(element); } else { int start = writeMap((MapColumnVector) vector, row); - writer.setOffsetAndSize(element, start, holder.cursor - start); + writer.setOffsetAndSizeFromPreviousCursor(element, start); } } } - static Converter buildConverter(BufferHolder holder, TypeDescription schema) { + static Converter buildConverter(final UnsafeWriter writer, final TypeDescription schema) { switch (schema.getCategory()) { case BOOLEAN: return new BooleanConverter(); @@ -856,13 +809,13 @@ static Converter buildConverter(BufferHolder holder, TypeDescription schema) { case STRING: case CHAR: case VARCHAR: - return new BinaryConverter(holder); + return new BinaryConverter(); case STRUCT: - return new StructConverter(holder, schema); + return new StructConverter(writer, schema); case LIST: - return new ListConverter(holder, schema); + return new ListConverter(writer, schema); case MAP: - return new MapConverter(holder, schema); + return new MapConverter(writer, schema); default: throw new IllegalArgumentException("Unhandled type " + schema); } diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java similarity index 98% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcWriter.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index 175be10483ce..68f566f16e8b 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -14,11 +14,11 @@ * limitations under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.orc.OrcFileAppender; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.OrcFileAppender; import org.apache.orc.TypeDescription; import org.apache.orc.storage.common.type.HiveDecimal; import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index a74d9cd43703..b5a48401521b 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -59,6 +59,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.spark.data.SparkOrcReader; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.AttributeReference; @@ -433,6 +434,10 @@ private Iterator open(FileScanTask task, Schema readSchema) { iter = newAvroIterable(location, task, readSchema); break; + case ORC: + iter = newOrcIterable(location, task, readSchema); + break; + default: throw new UnsupportedOperationException( "Cannot read unknown format: " + task.file().format()); @@ -444,8 +449,8 @@ private Iterator open(FileScanTask task, Schema readSchema) { } private CloseableIterable newAvroIterable(InputFile location, - FileScanTask task, - Schema readSchema) { + FileScanTask task, + Schema readSchema) { return Avro.read(location) .reuseContainers() .project(readSchema) @@ -455,8 +460,8 @@ private CloseableIterable newAvroIterable(InputFile location, } private CloseableIterable newParquetIterable(InputFile location, - FileScanTask task, - Schema readSchema) { + FileScanTask task, + Schema readSchema) { return Parquet.read(location) .project(readSchema) .split(task.start(), task.length()) @@ -465,6 +470,23 @@ private CloseableIterable newParquetIterable(InputFile location, .caseSensitive(caseSensitive) .build(); } + + private CloseableIterable newOrcIterable(InputFile location, + FileScanTask task, + Schema readSchema) { + final SparkOrcReader orcReader = new SparkOrcReader(location, task, readSchema); + return new CloseableIterable() { + @Override + public void close() throws IOException { + orcReader.close(); + } + + @Override + public Iterator iterator() { + return orcReader; + } + }; + } } private static class PartitionRowConverter implements Function { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java similarity index 82% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcScan.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java index 4a6fb269b441..9029dd957f2b 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcScan.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java @@ -14,25 +14,25 @@ * limitations under the License. */ -package com.netflix.iceberg.spark.source; - -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.orc.ORC; -import com.netflix.iceberg.orc.OrcFileAppender; -import com.netflix.iceberg.spark.data.AvroDataTest; -import com.netflix.iceberg.spark.data.RandomData; -import com.netflix.iceberg.spark.data.SparkOrcWriter; -import com.netflix.iceberg.spark.data.TestHelpers; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +package org.apache.iceberg.spark.source; + +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.orc.OrcFileAppender; +import org.apache.iceberg.spark.data.AvroDataTest; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkOrcWriter; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; @@ -58,7 +58,7 @@ import java.util.List; import java.util.UUID; -import static com.netflix.iceberg.Files.localOutput; +import static org.apache.iceberg.Files.localOutput; public class TestOrcScan extends AvroDataTest { private static final Configuration CONF = new Configuration(); From 0e72ec0a16bccd2cf859cd2cad293136dc2d9ae4 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Tue, 19 Mar 2019 12:17:52 -0700 Subject: [PATCH 03/23] Fix integration with spark 2.4.0 Unit tests for reads are passing. --- .../iceberg/spark/data/SparkOrcReader.java | 14 +-- .../iceberg/spark/data/SparkOrcWriter.java | 3 - .../apache/iceberg/spark/source/Writer.java | 34 ++++++ .../iceberg/spark/source/TestOrcWrite.java | 110 ++++++++++++++++++ .../iceberg/spark/source/TestOrcScan.java | 17 +-- 5 files changed, 154 insertions(+), 24 deletions(-) rename spark/src/main/java/{com/netflix => org/apache}/iceberg/spark/data/SparkOrcReader.java (98%) create mode 100644 spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcWrite.java diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java similarity index 98% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index ce7ce8f4dcf4..a13ea5cff7d4 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -64,7 +64,6 @@ public class SparkOrcReader implements Iterator, Closeable { private final static int INITIAL_SIZE = 128 * 1024; private final OrcIterator reader; - private final TypeDescription orcSchema; private final UnsafeRowWriter writer; private int nextRow = 0; private VectorizedRowBatch current = null; @@ -74,7 +73,7 @@ public SparkOrcReader(InputFile location, FileScanTask task, Schema readSchema) { ColumnIdMap columnIds = new ColumnIdMap(); - orcSchema = TypeConversion.toOrc(readSchema, columnIds); + TypeDescription orcSchema = TypeConversion.toOrc(readSchema, columnIds); reader = ORC.read(location) .split(task.start(), task.length()) .schema(readSchema) @@ -98,9 +97,9 @@ public UnsafeRow next() { current = reader.next(); nextRow = 0; } - // Reset the holder to start the buffer over again. - // BufferHolder.reset does the wrong thing... + writer.reset(); + writer.zeroOutNullBytes(); for(int c=0; c < current.cols.length; ++c) { converter[c].convert(writer, c, current.cols[c], nextRow); } @@ -219,7 +218,8 @@ private static void printRow(SpecializedGetters row, int ord, TypeDescription sc throw new IllegalArgumentException("Unhandled type " + schema); } } - static int getArrayElementSize(TypeDescription type) { + + private static int getArrayElementSize(TypeDescription type) { switch (type.getCategory()) { case BOOLEAN: case BYTE: @@ -618,7 +618,7 @@ private static class StructConverter implements Converter { int writeStruct(StructColumnVector vector, int row) { int start = childWriter.cursor(); - childWriter.reset(); + childWriter.resetRowWriter(); for(int c=0; c < children.length; ++c) { children[c].convert(childWriter, c, vector.fields[c], row); } @@ -669,7 +669,7 @@ int writeList(ListColumnVector v, int row) { int length = (int) v.lengths[row]; int start = childWriter.cursor(); childWriter.initialize(length); - for(int c=0; c < length; ++c) { + for(int c = 0; c < length; ++c) { children.convert(childWriter, c, v.child, offset + c); } return start; diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index 68f566f16e8b..9842545d5695 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -34,12 +34,9 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; -import java.sql.Timestamp; import java.util.List; /** diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java index 41241369a789..a7e2b419a113 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -24,6 +24,35 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.spark.data.SparkAvroWriter; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.spark.data.SparkOrcWriter; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; +import org.apache.spark.sql.sources.v2.writer.DataWriter; +import org.apache.spark.sql.sources.v2.writer.DataWriterFactory; +import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.Closeable; import java.io.IOException; import java.util.Arrays; @@ -226,6 +255,11 @@ public FileAppender newAppender(OutputFile file, FileFormat format) .schema(schema) .build(); + case ORC: + return new SparkOrcWriter( + ORC.write(file) + .schema(schema).build()); + default: throw new UnsupportedOperationException("Cannot write unknown format: " + format); } diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcWrite.java b/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcWrite.java new file mode 100644 index 000000000000..bc126709a3b3 --- /dev/null +++ b/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcWrite.java @@ -0,0 +1,110 @@ +/* + * 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 com.netflix.iceberg.spark.source; + +import com.google.common.collect.Lists; +import com.netflix.iceberg.FileFormat; +import com.netflix.iceberg.PartitionSpec; +import com.netflix.iceberg.Schema; +import com.netflix.iceberg.Table; +import com.netflix.iceberg.hadoop.HadoopTables; +import com.netflix.iceberg.types.Types; +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.CompressionKind; +import org.apache.orc.OrcConf; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +import static com.netflix.iceberg.types.Types.NestedField.optional; + +public class TestOrcWrite { + private static final Configuration CONF = new Configuration(); + private static final Schema SCHEMA = new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "data", Types.StringType.get()) + ); + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private static SparkSession spark = null; + + @BeforeClass + public static void startSpark() { + TestOrcWrite.spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @AfterClass + public static void stopSpark() { + SparkSession spark = TestOrcWrite.spark; + TestOrcWrite.spark = null; + spark.stop(); + } + + @Test + public void testBasicWrite() throws IOException { + File parent = temp.newFolder("orc"); + File location = new File(parent, "test"); + location.mkdirs(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + table.updateProperties() + .defaultFormat(FileFormat.ORC) + .set(OrcConf.COMPRESS.getAttribute(), CompressionKind.NONE.name()) + .commit(); + + List expected = Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c") + ); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + // TODO: incoming columns must be ordered according to the table's schema + df.select("id", "data").write() + .format("iceberg") + .mode("append") + .save(location.toString()); + + table.refresh(); + + Dataset result = spark.read() + .format("iceberg") + .load(location.toString()); + + List actual = result.orderBy("id").as( + Encoders.bean(SimpleRecord.class)).collectAsList(); + + Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); + Assert.assertEquals("Result rows should match", expected, actual); + } +} diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java index 9029dd957f2b..3ab299d90b82 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java @@ -24,26 +24,16 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.orc.OrcFileAppender; import org.apache.iceberg.spark.data.AvroDataTest; import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.data.SparkOrcWriter; import org.apache.iceberg.spark.data.TestHelpers; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; -import org.apache.orc.storage.serde2.io.DateWritable; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; -import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -52,8 +42,6 @@ import java.io.File; import java.io.IOException; -import java.sql.Date; -import java.sql.Timestamp; import java.util.Iterator; import java.util.List; import java.util.UUID; @@ -129,8 +117,9 @@ protected void writeAndValidate(Schema schema) throws IOException { Assert.assertEquals("Wrong number of rows", ROW_COUNT, rows.size()); Iterator expected = RandomData.generateSpark(tableSchema, ROW_COUNT, SEED); - for(int i=0; i < ROW_COUNT; ++i) { - TestHelpers.assertEquals("row " + i, schema.asStruct(), expected.next(), + for(int i = 0; i < ROW_COUNT; ++i) { + final InternalRow expectedRow = expected.next(); // useful for debug + TestHelpers.assertEquals("row " + i, schema.asStruct(), expectedRow, rows.get(i)); } } From 7b50e13f4a6093e1f0ef13b9c3dbf476c233658d Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Tue, 19 Mar 2019 18:45:18 -0700 Subject: [PATCH 04/23] Adjust for latest changes --- .../java/org/apache/iceberg/spark/data/SparkOrcWriter.java | 5 +++++ .../java/org/apache/iceberg/spark/source/TestOrcScan.java | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index 9842545d5695..6bee74facfab 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -420,6 +420,11 @@ public Metrics metrics() { return writer.metrics(); } + @Override + public long length() { + return writer.length(); + } + @Override public void close() throws IOException { if (batch.size > 0) { diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java index 3ab299d90b82..1acacb414133 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java @@ -115,8 +115,8 @@ protected void writeAndValidate(Schema schema) throws IOException { List rows = df.collectAsList(); Assert.assertEquals("Wrong number of rows", ROW_COUNT, rows.size()); - Iterator expected = RandomData.generateSpark(tableSchema, - ROW_COUNT, SEED); + Iterator expected = RandomData.generateSpark(tableSchema, ROW_COUNT, SEED) + .iterator(); for(int i = 0; i < ROW_COUNT; ++i) { final InternalRow expectedRow = expected.next(); // useful for debug TestHelpers.assertEquals("row " + i, schema.asStruct(), expectedRow, From 7b094ee9ab16e020ab64e946437b744c61cd4479 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Mon, 25 Mar 2019 14:26:41 -0700 Subject: [PATCH 05/23] Rename test to new org.apache pkg --- .../iceberg/spark/source/TestOrcWrite.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) rename spark/src/test/java/{com/netflix => org/apache}/iceberg/spark/source/TestOrcWrite.java (90%) diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcWrite.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcWrite.java similarity index 90% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcWrite.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestOrcWrite.java index bc126709a3b3..09ed81c454ab 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestOrcWrite.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcWrite.java @@ -14,15 +14,15 @@ * limitations under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.Lists; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.types.Types; import org.apache.hadoop.conf.Configuration; import org.apache.orc.CompressionKind; import org.apache.orc.OrcConf; @@ -41,7 +41,7 @@ import java.io.IOException; import java.util.List; -import static com.netflix.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.optional; public class TestOrcWrite { private static final Configuration CONF = new Configuration(); From 2051372dd38606a5afc6c857a355c9bcc635ae78 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Fri, 5 Apr 2019 12:26:19 -0700 Subject: [PATCH 06/23] Convert printRow methods to converters to String --- .../iceberg/spark/data/SparkOrcReader.java | 94 +++++++++---------- 1 file changed, 44 insertions(+), 50 deletions(-) diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index a13ea5cff7d4..bb942291969e 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -112,89 +112,83 @@ public void close() throws IOException { reader.close(); } - private static void printRow(SpecializedGetters row, TypeDescription schema) { - List children = schema.getChildren(); - System.out.print("{"); + private static String rowToString(SpecializedGetters row, TypeDescription schema) { + final List children = schema.getChildren(); + final StringBuilder rowBuilder = new StringBuilder("{"); + for(int c = 0; c < children.size(); ++c) { - System.out.print("\"" + schema.getFieldNames().get(c) + "\": "); - printRow(row, c, children.get(c)); + rowBuilder.append("\""); + rowBuilder.append(schema.getFieldNames().get(c)); + rowBuilder.append("\": "); + rowBuilder.append(rowEntryToString(row, c, children.get(c))); } - System.out.print("}"); + rowBuilder.append("}"); + return rowBuilder.toString(); } - private static void printRow(SpecializedGetters row, int ord, TypeDescription schema) { + private static String rowEntryToString(SpecializedGetters row, int ord, TypeDescription schema) { switch (schema.getCategory()) { case BOOLEAN: - System.out.print(row.getBoolean(ord)); - break; + return Boolean.toString(row.getBoolean(ord)); case BYTE: - System.out.print(row.getByte(ord)); - break; + return Byte.toString(row.getByte(ord)); case SHORT: - System.out.print(row.getShort(ord)); - break; + return Short.toString(row.getShort(ord)); case INT: - System.out.print(row.getInt(ord)); - break; + return Integer.toString(row.getInt(ord)); case LONG: - System.out.print(row.getLong(ord)); - break; + return Long.toString(row.getLong(ord)); case FLOAT: - System.out.print(row.getFloat(ord)); - break; + return Float.toString(row.getFloat(ord)); case DOUBLE: - System.out.print(row.getDouble(ord)); - break; + return Double.toString(row.getDouble(ord)); case CHAR: case VARCHAR: case STRING: - System.out.print("\"" + row.getUTF8String(ord) + "\""); - break; + return "\"" + row.getUTF8String(ord) + "\""; case BINARY: { byte[] bin = row.getBinary(ord); + final StringBuilder binStr; if (bin == null) { - System.out.print("null"); + binStr = new StringBuilder("null"); } else { - System.out.print("["); + binStr = new StringBuilder("["); for (int i = 0; i < bin.length; ++i) { if (i != 0) { - System.out.print(", "); + binStr.append(", "); } int v = bin[i] & 0xff; if (v < 16) { - System.out.print("0" + Integer.toHexString(v)); + binStr.append("0"); + binStr.append(Integer.toHexString(v)); } else { - System.out.print(Integer.toHexString(v)); + binStr.append(Integer.toHexString(v)); } } - System.out.print("]"); + binStr.append("]"); } - break; + return binStr.toString(); } case DECIMAL: - System.out.print(row.getDecimal(ord, schema.getPrecision(), schema.getScale())); - break; + return row.getDecimal(ord, schema.getPrecision(), schema.getScale()).toString(); case DATE: - System.out.print("\"" + new DateWritable(row.getInt(ord)) + "\""); - break; + return "\"" + new DateWritable(row.getInt(ord)) + "\""; case TIMESTAMP: - System.out.print("\"" + new Timestamp(row.getLong(ord)) + "\""); - break; + return "\"" + new Timestamp(row.getLong(ord)) + "\""; case STRUCT: - printRow(row.getStruct(ord, schema.getChildren().size()), schema); - break; + return rowToString(row.getStruct(ord, schema.getChildren().size()), schema); case LIST: { TypeDescription child = schema.getChildren().get(0); - System.out.print("["); + final StringBuilder listStr = new StringBuilder("["); ArrayData list = row.getArray(ord); for(int e=0; e < list.numElements(); ++e) { if (e != 0) { - System.out.print(", "); + listStr.append(", "); } - printRow(list, e, child); + listStr.append(rowEntryToString(list, e, child)); } - System.out.print("]"); - break; + listStr.append("]"); + return listStr.toString(); } case MAP: { TypeDescription keyType = schema.getChildren().get(0); @@ -202,17 +196,17 @@ private static void printRow(SpecializedGetters row, int ord, TypeDescription sc MapData map = row.getMap(ord); ArrayData keys = map.keyArray(); ArrayData values = map.valueArray(); - System.out.print("["); + StringBuilder mapStr = new StringBuilder("["); for(int e=0; e < map.numElements(); ++e) { if (e != 0) { - System.out.print(", "); + mapStr.append(", "); } - printRow(keys, e, keyType); - System.out.print(": "); - printRow(values, e, valueType); + mapStr.append(rowEntryToString(keys, e, keyType)); + mapStr.append(": "); + mapStr.append(rowEntryToString(values, e, valueType)); } - System.out.print("]"); - break; + mapStr.append("]"); + return mapStr.toString(); } default: throw new IllegalArgumentException("Unhandled type " + schema); From 278eecad8c87045325e458411a099c10b848c654 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Fri, 5 Apr 2019 14:49:04 -0700 Subject: [PATCH 07/23] Use ORC 1.5.0 with storage-api 2.6.0 Remove hack for converting decimal to long value. --- build.gradle | 2 +- .../iceberg/spark/data/SparkOrcReader.java | 16 ++-------------- 2 files changed, 3 insertions(+), 15 deletions(-) diff --git a/build.gradle b/build.gradle index 32f2e07355a9..0e6464109c76 100644 --- a/build.gradle +++ b/build.gradle @@ -76,7 +76,7 @@ subprojects { ext { hadoopVersion = '2.7.3' avroVersion = '1.8.2' - orcVersion = '1.4.2' + orcVersion = '1.5.0' parquetVersion = '1.10.0' hiveVersion = '1.2.1' diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index bb942291969e..cf59cc0172ad 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -500,19 +500,7 @@ public void convert(UnsafeArrayWriter writer, int element, ColumnVector vector, } } - /** - * 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. - */ - static class DecimalHack extends FastHiveDecimal { - long unscaledLong(FastHiveDecimal value) { - fastSet(value); - return fastSignum * fast1 * 10_000_000_000_000_000L + fast0; - } - } - private static class Decimal18Converter implements Converter { - final DecimalHack hack = new DecimalHack(); final int precision; final int scale; @@ -532,7 +520,7 @@ public void convert(UnsafeRowWriter writer, int column, ColumnVector vector, } else { HiveDecimalWritable v = ((DecimalColumnVector) vector).vector[row]; writer.write(column, - new Decimal().set(hack.unscaledLong(v), precision, v.scale()), + new Decimal().set(v.serialize64(v.scale()), v.precision(), v.scale()), precision, scale); } } @@ -548,7 +536,7 @@ public void convert(UnsafeArrayWriter writer, int element, } else { HiveDecimalWritable v = ((DecimalColumnVector) vector).vector[row]; writer.write(element, - new Decimal().set(hack.unscaledLong(v), precision, v.scale()), + new Decimal().set(v.serialize64(v.scale()), v.precision(), v.scale()), precision, scale); } } From 9bbf86cd144d1d8aeef15a9200b02cad44e6fc56 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Fri, 5 Apr 2019 14:50:37 -0700 Subject: [PATCH 08/23] Remove unused imports --- .../apache/iceberg/spark/source/Writer.java | 27 ------------------- 1 file changed, 27 deletions(-) diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java index a7e2b419a113..59a9ac3f978e 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -61,33 +61,6 @@ import java.util.Set; import java.util.UUID; import java.util.function.Function; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.exceptions.RuntimeIOException; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.LocationProvider; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.spark.data.SparkAvroWriter; -import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.iceberg.util.Tasks; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; -import org.apache.spark.sql.sources.v2.writer.DataWriter; -import org.apache.spark.sql.sources.v2.writer.DataWriterFactory; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.transform; From 5bdd32f3f749cc8e800489e5392a94407b664432 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Fri, 5 Apr 2019 14:54:02 -0700 Subject: [PATCH 09/23] Revert non-functional changes --- .../org/apache/iceberg/spark/data/SparkOrcReader.java | 1 - .../main/java/org/apache/iceberg/spark/source/Reader.java | 8 ++++---- .../main/java/org/apache/iceberg/spark/source/Writer.java | 6 +++--- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index cf59cc0172ad..b7d20a92acdd 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -24,7 +24,6 @@ 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; diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index b5a48401521b..8302f19f5d1c 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -449,8 +449,8 @@ private Iterator open(FileScanTask task, Schema readSchema) { } private CloseableIterable newAvroIterable(InputFile location, - FileScanTask task, - Schema readSchema) { + FileScanTask task, + Schema readSchema) { return Avro.read(location) .reuseContainers() .project(readSchema) @@ -460,8 +460,8 @@ private CloseableIterable newAvroIterable(InputFile location, } private CloseableIterable newParquetIterable(InputFile location, - FileScanTask task, - Schema readSchema) { + FileScanTask task, + Schema readSchema) { return Parquet.read(location) .project(readSchema) .split(task.start(), task.length()) diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java index 59a9ac3f978e..fb5f36fda92d 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -28,16 +28,16 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.io.FileIO; import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.orc.ORC; From f0adb15e0e190abe8b3166871c2e7866b4bd3371 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Mon, 15 Apr 2019 22:54:44 -0700 Subject: [PATCH 10/23] Refactor ORC write integration --- .../main/java/org/apache/iceberg/orc/ORC.java | 22 +++++- .../apache/iceberg/orc/OrcFileAppender.java | 77 ++++++++++++++----- .../apache/iceberg/orc/OrcValueWriter.java | 20 +++++ .../iceberg/spark/data/SparkOrcWriter.java | 54 +++---------- .../apache/iceberg/spark/source/Writer.java | 8 +- .../spark/data/TestSparkOrcReader.java | 32 ++++++++ .../iceberg/spark/source/TestOrcScan.java | 7 +- 7 files changed, 148 insertions(+), 72 deletions(-) create mode 100644 orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java create mode 100644 spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 157a761f1752..6d27d3079a28 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -21,11 +21,13 @@ import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; +import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.Schema; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.HadoopOutputFile; +import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.orc.OrcFile; @@ -44,6 +46,7 @@ public static class WriteBuilder { private final OutputFile file; private final Configuration conf; private Schema schema = null; + private Function> createWriterFunc; private Map metadata = new HashMap<>(); private WriteBuilder(OutputFile file) { @@ -65,15 +68,26 @@ public WriteBuilder config(String property, String value) { return this; } + public WriteBuilder createWriterFunc(Function> writerFunction) { + this.createWriterFunc = writerFunction; + return this; + } + + public WriteBuilder setAll(Map properties) { + properties.forEach(conf::set); + return this; + } + public WriteBuilder schema(Schema schema) { this.schema = schema; return this; } - public OrcFileAppender build() { - OrcFile.WriterOptions options = - OrcFile.writerOptions(conf); - return new OrcFileAppender(schema, file, options, metadata); + public FileAppender build() { + Preconditions.checkNotNull(schema, "Schema is required"); + OrcFile.WriterOptions options = OrcFile.writerOptions(conf); + return new OrcFileAppender<>(TypeConversion.toOrc(schema, new ColumnIdMap()), + this.file, createWriterFunc, options, metadata); } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index 257e0848f2aa..7159bc700860 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -20,9 +20,9 @@ import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; +import java.util.function.Function; import org.apache.hadoop.fs.Path; import org.apache.iceberg.Metrics; -import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.OutputFile; import org.apache.orc.ColumnStatistics; @@ -34,36 +34,39 @@ /** * Create a file appender for ORC. */ -public class OrcFileAppender implements FileAppender { - private final Writer writer; +public class OrcFileAppender implements FileAppender { + private final static int BATCH_SIZE = 1024; + private final TypeDescription orcSchema; private final ColumnIdMap columnIds = new ColumnIdMap(); private final Path path; + private final Writer writer; + private final VectorizedRowBatch batch; + private final OrcValueWriter valueWriter; private boolean isClosed = false; public static final String COLUMN_NUMBERS_ATTRIBUTE = "iceberg.column.ids"; - OrcFileAppender(Schema schema, - OutputFile file, - OrcFile.WriterOptions options, - Map metadata) { - orcSchema = TypeConversion.toOrc(schema, columnIds); - options.setSchema(orcSchema); + OrcFileAppender(TypeDescription schema, OutputFile file, + Function> createWriterFunc, + OrcFile.WriterOptions options, Map metadata) { + orcSchema = schema; path = new Path(file.location()); - try { - writer = OrcFile.createWriter(path, options); - } catch (IOException e) { - throw new RuntimeException("Can't create file " + path, e); - } - writer.addUserMetadata(COLUMN_NUMBERS_ATTRIBUTE, columnIds.serialize()); - metadata.forEach( - (key,value) -> writer.addUserMetadata(key, ByteBuffer.wrap(value))); + batch = orcSchema.createRowBatch(BATCH_SIZE); + + options.setSchema(orcSchema); + writer = newOrcWriter(file, columnIds, options, metadata); + valueWriter = newOrcValueWriter(orcSchema, createWriterFunc); } @Override - public void add(VectorizedRowBatch datum) { + public void add(D datum) { try { - writer.addRowBatch(datum); + valueWriter.write(datum, batch); + if (batch.size == BATCH_SIZE) { + writer.addRowBatch(batch); + batch.reset(); + } } catch (IOException e) { throw new RuntimeException("Problem writing to ORC file " + path, e); } @@ -108,12 +111,44 @@ public long length() { @Override public void close() throws IOException { if (!isClosed) { - this.isClosed = true; - writer.close(); + try { + if (batch.size > 0) { + writer.addRowBatch(batch); + batch.reset(); + } + } + finally { + writer.close(); + this.isClosed = true; + } } } public TypeDescription getSchema() { return orcSchema; } + + private static Writer newOrcWriter(OutputFile file, + ColumnIdMap columnIds, + OrcFile.WriterOptions options, Map metadata) { + final Path locPath = new Path(file.location()); + final Writer writer; + + try { + writer = OrcFile.createWriter(locPath, options); + } catch (IOException e) { + throw new RuntimeException("Can't create file " + locPath, e); + } + + writer.addUserMetadata(COLUMN_NUMBERS_ATTRIBUTE, columnIds.serialize()); + metadata.forEach((key,value) -> writer.addUserMetadata(key, ByteBuffer.wrap(value))); + + return writer; + } + + @SuppressWarnings("unchecked") + private static OrcValueWriter newOrcValueWriter(TypeDescription schema, + Function> createWriterFunc) { + return (OrcValueWriter) createWriterFunc.apply(schema); + } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java b/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java new file mode 100644 index 000000000000..5bda6052681a --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java @@ -0,0 +1,20 @@ +package org.apache.iceberg.orc; + +import java.io.IOException; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + +/** + * Write data value of a schema. + * @author Edgar Rodriguez-Diaz + * @since + */ +public interface OrcValueWriter { + + /** + * Writes the data. + * @param value the data value to write. + * @param output the VectorizedRowBatch to which the output will be written. + * @throws IOException if there's any IO error while writing the data value. + */ + void write(T value, VectorizedRowBatch output) throws IOException; +} diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index 6bee74facfab..2e3cf32a0296 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -16,9 +16,7 @@ package org.apache.iceberg.spark.data; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.orc.OrcFileAppender; +import org.apache.iceberg.orc.OrcValueWriter; import org.apache.orc.TypeDescription; import org.apache.orc.storage.common.type.HiveDecimal; import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; @@ -36,26 +34,28 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; -import java.io.IOException; import java.util.List; /** * This class acts as an adaptor from an OrcFileAppender to a * FileAppender<InternalRow>. */ -public class SparkOrcWriter implements FileAppender { - private final static int BATCH_SIZE = 1024; - private final VectorizedRowBatch batch; - private final OrcFileAppender writer; +public class SparkOrcWriter implements OrcValueWriter { + private final Converter[] converters; - public SparkOrcWriter(OrcFileAppender writer) { - TypeDescription schema = writer.getSchema(); - batch = schema.createRowBatch(BATCH_SIZE); - this.writer = writer; + public SparkOrcWriter(TypeDescription schema) { converters = buildConverters(schema); } + @Override + public void write(InternalRow value, VectorizedRowBatch output) { + int row = output.size++; + for(int c=0; c < converters.length; ++c) { + converters[c].addValue(row, c, value, output.cols[c]); + } + } + /** * The interface for the conversion from Spark's SpecializedGetters to * ORC's ColumnVectors. @@ -403,34 +403,4 @@ private static Converter[] buildConverters(TypeDescription schema) { return result; } - @Override - public void add(InternalRow datum) { - int row = batch.size++; - for(int c=0; c < converters.length; ++c) { - converters[c].addValue(row, c, datum, batch.cols[c]); - } - if (batch.size == BATCH_SIZE) { - writer.add(batch); - batch.reset(); - } - } - - @Override - public Metrics metrics() { - return writer.metrics(); - } - - @Override - public long length() { - return writer.length(); - } - - @Override - public void close() throws IOException { - if (batch.size > 0) { - writer.add(batch); - batch.reset(); - } - writer.close(); - } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java index fb5f36fda92d..afd6a0bfa719 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -229,9 +229,11 @@ public FileAppender newAppender(OutputFile file, FileFormat format) .build(); case ORC: - return new SparkOrcWriter( - ORC.write(file) - .schema(schema).build()); + return ORC.write(file) + .schema(schema) + .setAll(properties) + .createWriterFunc(SparkOrcWriter::new) + .build(); default: throw new UnsupportedOperationException("Cannot write unknown format: " + format); diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java new file mode 100644 index 000000000000..d5c7ce98aa23 --- /dev/null +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -0,0 +1,32 @@ +package org.apache.iceberg.spark.data; + +import java.io.File; +import java.io.IOException; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.Assert; + +/** + * @author Edgar Rodriguez-Diaz + * @since + */ +public class TestSparkOrcReader extends AvroDataTest { + @Override + protected void writeAndValidate(Schema schema) throws IOException { + final Iterable expected = RandomData + .generateSpark(schema, 100, 0L); + + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + try (FileAppender writer = ORC.write(Files.localOutput(testFile)) + .createWriterFunc(SparkOrcWriter::new) + .schema(schema) + .build()) { + writer.addAll(expected); + } + } +} diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java index 1acacb414133..cced72c43da7 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java @@ -24,7 +24,9 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.orc.OrcFileAppender; import org.apache.iceberg.spark.data.AvroDataTest; import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.data.SparkOrcWriter; @@ -90,9 +92,10 @@ protected void writeAndValidate(Schema schema) throws IOException { Schema tableSchema = table.schema(); Metrics metrics; - SparkOrcWriter writer = new SparkOrcWriter(ORC.write(localOutput(orcFile)) + FileAppender writer = ORC.write(localOutput(orcFile)) .schema(tableSchema) - .build()); + .createWriterFunc(SparkOrcWriter::new) + .build(); try { writer.addAll(RandomData.generateSpark(tableSchema, ROW_COUNT, SEED)); } finally { From 7bf13ee68532b8210d0bf389edb80d2d5bee18c9 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Wed, 24 Apr 2019 00:32:19 -0700 Subject: [PATCH 11/23] Refactor organization of ORC package to match data source API in Iceberg --- .../main/java/org/apache/iceberg/orc/ORC.java | 25 ++--- .../org/apache/iceberg/orc/OrcIterable.java | 101 ++++++++++++++++++ .../apache/iceberg/orc/OrcValueReader.java | 17 +++ ...r.java => VectorizedRowBatchIterator.java} | 11 +- .../iceberg/spark/data/SparkOrcReader.java | 74 +++++-------- .../apache/iceberg/spark/source/Reader.java | 18 ++-- .../iceberg/spark/data/TestHelpers.java | 5 + .../spark/data/TestSparkOrcReader.java | 24 ++++- 8 files changed, 193 insertions(+), 82 deletions(-) create mode 100644 orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java create mode 100644 orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java rename orc/src/main/java/org/apache/iceberg/orc/{OrcIterator.java => VectorizedRowBatchIterator.java} (86%) diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 6d27d3079a28..5f5ca5e59494 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -27,6 +27,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.HadoopOutputFile; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; @@ -102,6 +103,8 @@ public static class ReadBuilder { private Long start = null; private Long length = null; + private Function> readerFunction; + private ReadBuilder(InputFile file) { Preconditions.checkNotNull(file, "Input file cannot be null"); this.file = file; @@ -135,22 +138,14 @@ public ReadBuilder config(String property, String value) { return this; } - public OrcIterator build() { + public ReadBuilder createReaderFunc(Function> readerFunction) { + this.readerFunction = readerFunction; + return this; + } + + public CloseableIterable build() { Preconditions.checkNotNull(schema, "Schema is required"); - try { - Path path = new Path(file.location()); - Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf)); - ColumnIdMap columnIds = new ColumnIdMap(); - TypeDescription orcSchema = TypeConversion.toOrc(schema, columnIds); - Reader.Options options = reader.options(); - if (start != null) { - options.range(start, length); - } - options.schema(orcSchema); - return new OrcIterator(path, orcSchema, reader.rows(options)); - } catch (IOException e) { - throw new RuntimeException("Can't open " + file.location(), e); - } + return new OrcIterable<>(file, conf, schema, start, length, readerFunction); } } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java new file mode 100644 index 000000000000..56ebad31bcec --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java @@ -0,0 +1,101 @@ +package org.apache.iceberg.orc; + +import java.io.IOException; +import java.util.Iterator; +import java.util.function.Function; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + +/** + * @author Edgar Rodriguez-Diaz + * @since + */ +public class OrcIterable extends CloseableGroup implements CloseableIterable { + private final Schema schema; + private final Function> readerFunction; + private final VectorizedRowBatchIterator orcIter; + + public OrcIterable(InputFile file, Configuration config, Schema schema, + Long start, Long length, + Function> readerFunction) { + this.schema = schema; + this.readerFunction = readerFunction; + final Reader orcFileReader = newFileReader(file, config); + this.orcIter = newOrcIterator(file, TypeConversion.toOrc(schema, new ColumnIdMap()), + start, length, orcFileReader); + } + + @SuppressWarnings("unchecked") + @Override + public Iterator iterator() { + return new OrcIterator(orcIter, (OrcValueReader) readerFunction.apply(schema)); + } + + private static VectorizedRowBatchIterator newOrcIterator(InputFile file, TypeDescription orcSchema, + Long start, Long length, + Reader orcFileReader) { + final Reader.Options options = orcFileReader.options(); + if (start != null) { + options.range(start, length); + } + options.schema(orcSchema); + + try { + return new VectorizedRowBatchIterator(file.location(), orcSchema, orcFileReader.rows(options)); + } + catch (IOException ioe) { + throw new RuntimeIOException(ioe, "Failed to get ORC rows for file: %s", file); + } + } + + private static Reader newFileReader(InputFile file, Configuration config) { + try { + return OrcFile.createReader(new Path(file.location()), + OrcFile.readerOptions(config)); + } + catch (IOException ioe) { + throw new RuntimeIOException(ioe, "Failed to open file: %s", file); + } + } + + private class OrcIterator implements Iterator { + + private int nextRow; + private VectorizedRowBatch current; + + final VectorizedRowBatchIterator batchIter; + final OrcValueReader reader; + + OrcIterator(VectorizedRowBatchIterator batchIter, OrcValueReader reader) { + this.batchIter = batchIter; + this.reader = reader; + current = null; + nextRow = 0; + } + + @Override + public boolean hasNext() { + return (current != null && nextRow < current.size) || batchIter.hasNext(); + } + + @Override + public T next() { + if (current == null || nextRow >= current.size) { + current = batchIter.next(); + nextRow = 0; + } + + return this.reader.read(current, nextRow++); + } + } + +} diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java new file mode 100644 index 000000000000..b2608af8b77e --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java @@ -0,0 +1,17 @@ +package org.apache.iceberg.orc; + +/** + * @author Edgar Rodriguez-Diaz + * @since + */ +public interface OrcValueReader { + + /** + * Reads + * @param reuse + * @param row + * @return + */ + T read(Object reuse, int row); + +} diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcIterator.java b/orc/src/main/java/org/apache/iceberg/orc/VectorizedRowBatchIterator.java similarity index 86% rename from orc/src/main/java/org/apache/iceberg/orc/OrcIterator.java rename to orc/src/main/java/org/apache/iceberg/orc/VectorizedRowBatchIterator.java index 589e5eea410f..b586a589b585 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcIterator.java +++ b/orc/src/main/java/org/apache/iceberg/orc/VectorizedRowBatchIterator.java @@ -19,7 +19,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.Iterator; -import org.apache.hadoop.fs.Path; import org.apache.orc.RecordReader; import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; @@ -29,14 +28,14 @@ * Because the same VectorizedRowBatch is reused on each call to next, * it gets changed when hasNext or next is called. */ -public class OrcIterator implements Iterator, Closeable { - private final Path filename; +public class VectorizedRowBatchIterator implements Iterator, Closeable { + private final String fileLocation; private final RecordReader rows; private final VectorizedRowBatch batch; private boolean advanced = false; - OrcIterator(Path filename, TypeDescription schema, RecordReader rows) { - this.filename = filename; + VectorizedRowBatchIterator(String fileLocation, TypeDescription schema, RecordReader rows) { + this.fileLocation = fileLocation; this.rows = rows; this.batch = schema.createRowBatch(); } @@ -51,7 +50,7 @@ private void advance() { try { rows.nextBatch(batch); } catch (IOException e) { - throw new RuntimeException("Problem reading ORC file " + filename, e); + throw new RuntimeException("Problem reading ORC file " + fileLocation, e); } advanced = true; } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index b7d20a92acdd..41e5c4582a96 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -16,12 +16,9 @@ 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.OrcValueReader; import org.apache.iceberg.orc.TypeConversion; import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; @@ -38,7 +35,6 @@ 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; @@ -47,11 +43,8 @@ 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; /** @@ -60,55 +53,40 @@ * * It minimizes allocations by reusing most of the objects in the implementation. */ -public class SparkOrcReader implements Iterator, Closeable { +public class SparkOrcReader implements OrcValueReader { 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)); - } + private final int numFields; + private final TypeDescription orcSchema; + + public SparkOrcReader(Schema readSchema) { + orcSchema = TypeConversion.toOrc(readSchema, new ColumnIdMap()); + numFields = readSchema.columns().size(); } - @Override - public boolean hasNext() { - return (current != null && nextRow < current.size) || reader.hasNext(); + private Converter[] buildConverters(final UnsafeRowWriter writer) { + final Converter[] converters = new Converter[numFields]; + for(int c = 0; c < numFields; ++c) { + converters[c] = buildConverter(writer, orcSchema.getChildren().get(c)); + } + return converters; } @Override - public UnsafeRow next() { - if (current == null || nextRow >= current.size) { - current = reader.next(); - nextRow = 0; + public InternalRow read(Object reuse, int row) { + if (!(reuse instanceof VectorizedRowBatch)) { + throw new IllegalArgumentException("Value to read must be of type "+ VectorizedRowBatch.class); } - 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(); - } + final VectorizedRowBatch batch = (VectorizedRowBatch) reuse; + final UnsafeRowWriter rowWriter = new UnsafeRowWriter(numFields, INITIAL_SIZE); + final Converter[] converters = buildConverters(rowWriter); - @Override - public void close() throws IOException { - reader.close(); + 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(); } private static String rowToString(SpecializedGetters row, TypeDescription schema) { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index 8302f19f5d1c..5772c6083ca4 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -51,6 +51,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -474,18 +475,11 @@ private CloseableIterable newParquetIterable(InputFile location, private CloseableIterable newOrcIterable(InputFile location, FileScanTask task, Schema readSchema) { - final SparkOrcReader orcReader = new SparkOrcReader(location, task, readSchema); - return new CloseableIterable() { - @Override - public void close() throws IOException { - orcReader.close(); - } - - @Override - public Iterator iterator() { - return orcReader; - } - }; + return ORC.read(location) + .schema(readSchema) + .split(task.start(), task.length()) + .createReaderFunc(SparkOrcReader::new) + .build(); } } diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index a760455ba076..96ec068abc67 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -41,6 +41,8 @@ import org.apache.orc.storage.serde2.io.DateWritable; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; import org.apache.spark.sql.catalyst.expressions.GenericRow; import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; import org.apache.spark.sql.catalyst.util.ArrayData; @@ -52,6 +54,7 @@ import org.apache.spark.sql.types.MapType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.BinaryType; import org.apache.spark.unsafe.types.UTF8String; import org.junit.Assert; import scala.collection.Seq; @@ -594,6 +597,8 @@ private static void assertEquals(String context, DataType type, Object expected, actual instanceof MapData); assertEquals(context, (MapType) type, (MapData) expected, (MapData) actual); + } else if (type instanceof BinaryType) { + assertEqualBytes(context, (byte[]) expected, (byte[]) actual); } else { Assert.assertEquals("Value should match expected: " + context, expected, actual); } diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java index d5c7ce98aa23..2a7d7f89ae08 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -1,12 +1,21 @@ package org.apache.iceberg.spark.data; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + import java.io.File; import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.avro.generic.GenericData; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.DataType; import org.junit.Assert; /** @@ -19,7 +28,7 @@ protected void writeAndValidate(Schema schema) throws IOException { final Iterable expected = RandomData .generateSpark(schema, 100, 0L); - File testFile = temp.newFile(); + final File testFile = temp.newFile(); Assert.assertTrue("Delete should succeed", testFile.delete()); try (FileAppender writer = ORC.write(Files.localOutput(testFile)) @@ -28,5 +37,18 @@ protected void writeAndValidate(Schema schema) throws IOException { .build()) { writer.addAll(expected); } + + try (CloseableIterable reader = ORC.read(Files.localInput(testFile)) + .schema(schema) + .createReaderFunc(SparkOrcReader::new) + .build()) { + final Iterator actualRows = reader.iterator(); + final Iterator expectedRows = expected.iterator(); + while (expectedRows.hasNext()) { + Assert.assertTrue("Should have expected number of rows", actualRows.hasNext()); + assertEquals(schema, expectedRows.next(), actualRows.next()); + } + Assert.assertFalse("Should not have extra rows", actualRows.hasNext()); + } } } From 0274109f8410556231ebb27654bc789d4dbce7c9 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Wed, 24 Apr 2019 00:37:45 -0700 Subject: [PATCH 12/23] Remove ORC tests using spark --- .../main/java/org/apache/iceberg/orc/ORC.java | 3 - .../apache/iceberg/spark/source/Writer.java | 9 -- .../iceberg/spark/data/TestHelpers.java | 2 - .../spark/data/TestSparkOrcReader.java | 9 -- .../iceberg/spark/source/TestOrcScan.java | 129 ------------------ .../iceberg/spark/source/TestOrcWrite.java | 110 --------------- 6 files changed, 262 deletions(-) delete mode 100644 spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java delete mode 100644 spark/src/test/java/org/apache/iceberg/spark/source/TestOrcWrite.java diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 5f5ca5e59494..c346b6746c64 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -17,13 +17,11 @@ package org.apache.iceberg.orc; import com.google.common.base.Preconditions; -import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; import java.util.function.Function; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.iceberg.Schema; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.HadoopOutputFile; @@ -32,7 +30,6 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.orc.OrcFile; -import org.apache.orc.Reader; import org.apache.orc.TypeDescription; public class ORC { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java index afd6a0bfa719..87e41605e97b 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -40,11 +40,9 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.spark.data.SparkAvroWriter; import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.iceberg.spark.data.SparkOrcWriter; import org.apache.iceberg.util.Tasks; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; @@ -228,13 +226,6 @@ public FileAppender newAppender(OutputFile file, FileFormat format) .schema(schema) .build(); - case ORC: - return ORC.write(file) - .schema(schema) - .setAll(properties) - .createWriterFunc(SparkOrcWriter::new) - .build(); - default: throw new UnsupportedOperationException("Cannot write unknown format: " + format); } diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 96ec068abc67..69d0d84c4cba 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -41,8 +41,6 @@ import org.apache.orc.storage.serde2.io.DateWritable; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; -import org.apache.spark.sql.catalyst.encoders.RowEncoder; import org.apache.spark.sql.catalyst.expressions.GenericRow; import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; import org.apache.spark.sql.catalyst.util.ArrayData; diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java index 2a7d7f89ae08..7ede1d562408 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -1,27 +1,18 @@ package org.apache.iceberg.spark.data; -import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; import java.io.File; import java.io.IOException; import java.util.Iterator; -import java.util.List; -import org.apache.avro.generic.GenericData; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.DataType; import org.junit.Assert; -/** - * @author Edgar Rodriguez-Diaz - * @since - */ public class TestSparkOrcReader extends AvroDataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java deleted file mode 100644 index cced72c43da7..000000000000 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcScan.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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.source; - -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.orc.OrcFileAppender; -import org.apache.iceberg.spark.data.AvroDataTest; -import org.apache.iceberg.spark.data.RandomData; -import org.apache.iceberg.spark.data.SparkOrcWriter; -import org.apache.iceberg.spark.data.TestHelpers; -import org.apache.hadoop.conf.Configuration; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import java.util.UUID; - -import static org.apache.iceberg.Files.localOutput; - -public class TestOrcScan extends AvroDataTest { - private static final Configuration CONF = new Configuration(); - - @Rule - public TemporaryFolder temp = new TemporaryFolder(); - - private static SparkSession spark = null; - - @BeforeClass - public static void startSpark() { - TestOrcScan.spark = SparkSession.builder().master("local[2]").getOrCreate(); - } - - @AfterClass - public static void stopSpark() { - SparkSession spark = TestOrcScan.spark; - TestOrcScan.spark = null; - spark.stop(); - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - System.out.println("Starting ORC test with " + schema); - final int ROW_COUNT = 100; - final long SEED = 1; - File parent = temp.newFolder("orc"); - File location = new File(parent, "test"); - File dataFolder = new File(location, "data"); - dataFolder.mkdirs(); - - File orcFile = new File(dataFolder, - FileFormat.ORC.addExtension(UUID.randomUUID().toString())); - - HadoopTables tables = new HadoopTables(CONF); - Table table = tables.create(schema, PartitionSpec.unpartitioned(), - location.toString()); - - // Important: use the table's schema for the rest of the test - // When tables are created, the column ids are reassigned. - Schema tableSchema = table.schema(); - - Metrics metrics; - FileAppender writer = ORC.write(localOutput(orcFile)) - .schema(tableSchema) - .createWriterFunc(SparkOrcWriter::new) - .build(); - try { - writer.addAll(RandomData.generateSpark(tableSchema, ROW_COUNT, SEED)); - } finally { - writer.close(); - // close writes the last batch, so metrics are not correct until after close is called - metrics = writer.metrics(); - } - - DataFile file = DataFiles.builder(PartitionSpec.unpartitioned()) - .withFileSizeInBytes(orcFile.length()) - .withPath(orcFile.toString()) - .withMetrics(metrics) - .build(); - - table.newAppend().appendFile(file).commit(); - - Dataset df = spark.read() - .format("iceberg") - .load(location.toString()); - - List rows = df.collectAsList(); - Assert.assertEquals("Wrong number of rows", ROW_COUNT, rows.size()); - Iterator expected = RandomData.generateSpark(tableSchema, ROW_COUNT, SEED) - .iterator(); - for(int i = 0; i < ROW_COUNT; ++i) { - final InternalRow expectedRow = expected.next(); // useful for debug - TestHelpers.assertEquals("row " + i, schema.asStruct(), expectedRow, - rows.get(i)); - } - } -} diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcWrite.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcWrite.java deleted file mode 100644 index 09ed81c454ab..000000000000 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestOrcWrite.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.source; - -import com.google.common.collect.Lists; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.types.Types; -import org.apache.hadoop.conf.Configuration; -import org.apache.orc.CompressionKind; -import org.apache.orc.OrcConf; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.IOException; -import java.util.List; - -import static org.apache.iceberg.types.Types.NestedField.optional; - -public class TestOrcWrite { - private static final Configuration CONF = new Configuration(); - private static final Schema SCHEMA = new Schema( - optional(1, "id", Types.IntegerType.get()), - optional(2, "data", Types.StringType.get()) - ); - - @Rule - public TemporaryFolder temp = new TemporaryFolder(); - - private static SparkSession spark = null; - - @BeforeClass - public static void startSpark() { - TestOrcWrite.spark = SparkSession.builder().master("local[2]").getOrCreate(); - } - - @AfterClass - public static void stopSpark() { - SparkSession spark = TestOrcWrite.spark; - TestOrcWrite.spark = null; - spark.stop(); - } - - @Test - public void testBasicWrite() throws IOException { - File parent = temp.newFolder("orc"); - File location = new File(parent, "test"); - location.mkdirs(); - - HadoopTables tables = new HadoopTables(CONF); - PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); - Table table = tables.create(SCHEMA, spec, location.toString()); - table.updateProperties() - .defaultFormat(FileFormat.ORC) - .set(OrcConf.COMPRESS.getAttribute(), CompressionKind.NONE.name()) - .commit(); - - List expected = Lists.newArrayList( - new SimpleRecord(1, "a"), - new SimpleRecord(2, "b"), - new SimpleRecord(3, "c") - ); - - Dataset df = spark.createDataFrame(expected, SimpleRecord.class); - - // TODO: incoming columns must be ordered according to the table's schema - df.select("id", "data").write() - .format("iceberg") - .mode("append") - .save(location.toString()); - - table.refresh(); - - Dataset result = spark.read() - .format("iceberg") - .load(location.toString()); - - List actual = result.orderBy("id").as( - Encoders.bean(SimpleRecord.class)).collectAsList(); - - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); - } -} From 82aea489448509364a3f9259e16247c578eb0fb0 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Wed, 1 May 2019 11:17:39 -0700 Subject: [PATCH 13/23] Add License and fix JavaDocs --- .../org/apache/iceberg/orc/OrcIterable.java | 22 +++++++++++++-- .../apache/iceberg/orc/OrcValueReader.java | 27 ++++++++++++++----- .../apache/iceberg/orc/OrcValueWriter.java | 21 +++++++++++++-- .../spark/data/TestSparkOrcReader.java | 19 +++++++++++++ 4 files changed, 79 insertions(+), 10 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java index 56ebad31bcec..3ccc9d35c2aa 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java @@ -1,3 +1,22 @@ +/* + * 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 java.io.IOException; @@ -16,8 +35,7 @@ import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; /** - * @author Edgar Rodriguez-Diaz - * @since + * Iterable used to read rows from ORC. */ public class OrcIterable extends CloseableGroup implements CloseableIterable { private final Schema schema; diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java index b2608af8b77e..0b48b3ad6598 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java @@ -1,16 +1,31 @@ +/* + * 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; /** - * @author Edgar Rodriguez-Diaz - * @since + * Used for implementing ORC value readers. */ public interface OrcValueReader { /** - * Reads - * @param reuse - * @param row - * @return + * Reads a value in row. */ T read(Object reuse, int row); diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java b/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java index 5bda6052681a..5f1e167fa4b3 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java @@ -1,3 +1,22 @@ +/* + * 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 java.io.IOException; @@ -5,8 +24,6 @@ /** * Write data value of a schema. - * @author Edgar Rodriguez-Diaz - * @since */ public interface OrcValueWriter { diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java index 7ede1d562408..1a20ff8e59ba 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -1,3 +1,22 @@ +/* + * 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 static org.apache.iceberg.spark.data.TestHelpers.assertEquals; From 6c3173ce4f92133e1a82d430640056950f922052 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Wed, 1 May 2019 11:48:31 -0700 Subject: [PATCH 14/23] Code review changes in OrcIterable --- .../org/apache/iceberg/orc/OrcIterable.java | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java index 3ccc9d35c2aa..7f0994f8373a 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java @@ -38,24 +38,31 @@ * Iterable used to read rows from ORC. */ public class OrcIterable extends CloseableGroup implements CloseableIterable { + private final Configuration config; private final Schema schema; + private final InputFile file; + private final Long start; + private final Long length; private final Function> readerFunction; - private final VectorizedRowBatchIterator orcIter; public OrcIterable(InputFile file, Configuration config, Schema schema, Long start, Long length, Function> readerFunction) { this.schema = schema; this.readerFunction = readerFunction; - final Reader orcFileReader = newFileReader(file, config); - this.orcIter = newOrcIterator(file, TypeConversion.toOrc(schema, new ColumnIdMap()), - start, length, orcFileReader); + this.file = file; + this.start = start; + this.length = length; + this.config = config; } @SuppressWarnings("unchecked") @Override public Iterator iterator() { - return new OrcIterator(orcIter, (OrcValueReader) readerFunction.apply(schema)); + return new OrcIterator( + newOrcIterator(file, TypeConversion.toOrc(schema, new ColumnIdMap()), + start, length, newFileReader(file, config)), + readerFunction.apply(schema)); } private static VectorizedRowBatchIterator newOrcIterator(InputFile file, TypeDescription orcSchema, @@ -85,7 +92,7 @@ private static Reader newFileReader(InputFile file, Configuration config) { } } - private class OrcIterator implements Iterator { + private static class OrcIterator implements Iterator { private int nextRow; private VectorizedRowBatch current; From 87cf57cb7c2403f953bad4f63e1d9bd9c9ffc534 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Wed, 1 May 2019 12:38:11 -0700 Subject: [PATCH 15/23] Code style --- .../java/org/apache/iceberg/orc/OrcFileAppender.java | 7 +++---- .../main/java/org/apache/iceberg/orc/OrcIterable.java | 10 ++++------ .../org/apache/iceberg/spark/data/SparkOrcReader.java | 3 +-- 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index 7159bc700860..f068c47ad9fa 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -34,7 +34,7 @@ /** * Create a file appender for ORC. */ -public class OrcFileAppender implements FileAppender { +class OrcFileAppender implements FileAppender { private final static int BATCH_SIZE = 1024; private final TypeDescription orcSchema; @@ -45,7 +45,7 @@ public class OrcFileAppender implements FileAppender { private final OrcValueWriter valueWriter; private boolean isClosed = false; - public static final String COLUMN_NUMBERS_ATTRIBUTE = "iceberg.column.ids"; + static final String COLUMN_NUMBERS_ATTRIBUTE = "iceberg.column.ids"; OrcFileAppender(TypeDescription schema, OutputFile file, Function> createWriterFunc, @@ -116,8 +116,7 @@ public void close() throws IOException { writer.addRowBatch(batch); batch.reset(); } - } - finally { + } finally { writer.close(); this.isClosed = true; } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java index 7f0994f8373a..aeda7b5cd72e 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java @@ -37,7 +37,7 @@ /** * Iterable used to read rows from ORC. */ -public class OrcIterable extends CloseableGroup implements CloseableIterable { +class OrcIterable extends CloseableGroup implements CloseableIterable { private final Configuration config; private final Schema schema; private final InputFile file; @@ -45,7 +45,7 @@ public class OrcIterable extends CloseableGroup implements CloseableIterable< private final Long length; private final Function> readerFunction; - public OrcIterable(InputFile file, Configuration config, Schema schema, + OrcIterable(InputFile file, Configuration config, Schema schema, Long start, Long length, Function> readerFunction) { this.schema = schema; @@ -76,8 +76,7 @@ private static VectorizedRowBatchIterator newOrcIterator(InputFile file, TypeDes try { return new VectorizedRowBatchIterator(file.location(), orcSchema, orcFileReader.rows(options)); - } - catch (IOException ioe) { + } catch (IOException ioe) { throw new RuntimeIOException(ioe, "Failed to get ORC rows for file: %s", file); } } @@ -86,8 +85,7 @@ private static Reader newFileReader(InputFile file, Configuration config) { try { return OrcFile.createReader(new Path(file.location()), OrcFile.readerOptions(config)); - } - catch (IOException ioe) { + } catch (IOException ioe) { throw new RuntimeIOException(ioe, "Failed to open file: %s", file); } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index 41e5c4582a96..7e2077c4255e 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -417,8 +417,7 @@ 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; + return (vector.time[row]/1000) * 1_000_000 + vector.nanos[row] / 1000; } @Override From b5004a4e529777635bb5ea5b3c91e8d5a8b00855 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Wed, 1 May 2019 12:44:52 -0700 Subject: [PATCH 16/23] Use same license in the project --- .../org/apache/iceberg/orc/ColumnIdMap.java | 23 ++++++++++-------- .../main/java/org/apache/iceberg/orc/ORC.java | 23 ++++++++++-------- .../apache/iceberg/orc/OrcFileAppender.java | 24 +++++++++++-------- .../apache/iceberg/orc/TypeConversion.java | 23 ++++++++++-------- .../orc/VectorizedRowBatchIterator.java | 23 ++++++++++-------- .../iceberg/spark/data/SparkOrcReader.java | 23 ++++++++++-------- .../iceberg/spark/data/SparkOrcWriter.java | 23 ++++++++++-------- 7 files changed, 92 insertions(+), 70 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/ColumnIdMap.java b/orc/src/main/java/org/apache/iceberg/orc/ColumnIdMap.java index 330554f63e14..16dc3b0294ca 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ColumnIdMap.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ColumnIdMap.java @@ -1,17 +1,20 @@ /* - * 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 + * 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. + * 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; diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index c346b6746c64..1611eb49c7e9 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -1,17 +1,20 @@ /* - * 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 + * 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. + * 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; diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index f068c47ad9fa..440ddf1be5b2 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -1,18 +1,22 @@ /* - * 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 + * 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. + * 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.base.Preconditions; diff --git a/orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java b/orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java index bc57f8d03ebe..f9839f637cd5 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java +++ b/orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java @@ -1,17 +1,20 @@ /* - * 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 + * 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. + * 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; diff --git a/orc/src/main/java/org/apache/iceberg/orc/VectorizedRowBatchIterator.java b/orc/src/main/java/org/apache/iceberg/orc/VectorizedRowBatchIterator.java index b586a589b585..ddc0bce97ca4 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/VectorizedRowBatchIterator.java +++ b/orc/src/main/java/org/apache/iceberg/orc/VectorizedRowBatchIterator.java @@ -1,17 +1,20 @@ /* - * 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 + * 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. + * 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; diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index 7e2077c4255e..dc5530a53360 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -1,17 +1,20 @@ /* - * 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 + * 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. + * 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; diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index 2e3cf32a0296..80e287835198 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -1,17 +1,20 @@ /* - * 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 + * 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. + * 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; From 4b866728e0537b48779b13738aa64eda452cffd7 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Mon, 6 May 2019 17:10:17 -0700 Subject: [PATCH 17/23] Remove unnecessary code --- .../java/org/apache/iceberg/orc/OrcFileAppender.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index 440ddf1be5b2..6c283493098e 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -35,12 +35,12 @@ import org.apache.orc.Writer; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import static org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch.DEFAULT_SIZE; + /** * Create a file appender for ORC. */ class OrcFileAppender implements FileAppender { - private final static int BATCH_SIZE = 1024; - private final TypeDescription orcSchema; private final ColumnIdMap columnIds = new ColumnIdMap(); private final Path path; @@ -56,7 +56,7 @@ class OrcFileAppender implements FileAppender { OrcFile.WriterOptions options, Map metadata) { orcSchema = schema; path = new Path(file.location()); - batch = orcSchema.createRowBatch(BATCH_SIZE); + batch = orcSchema.createRowBatch(DEFAULT_SIZE); options.setSchema(orcSchema); writer = newOrcWriter(file, columnIds, options, metadata); @@ -67,7 +67,7 @@ class OrcFileAppender implements FileAppender { public void add(D datum) { try { valueWriter.write(datum, batch); - if (batch.size == BATCH_SIZE) { + if (batch.size == DEFAULT_SIZE) { writer.addRowBatch(batch); batch.reset(); } @@ -127,10 +127,6 @@ public void close() throws IOException { } } - public TypeDescription getSchema() { - return orcSchema; - } - private static Writer newOrcWriter(OutputFile file, ColumnIdMap columnIds, OrcFile.WriterOptions options, Map metadata) { From 2c8286d12fa7f1c90c43ed476fef3b1e09b9ddd8 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Mon, 6 May 2019 17:46:42 -0700 Subject: [PATCH 18/23] Allow configuration of vector batch size in OrcFileAppender --- orc/src/main/java/org/apache/iceberg/orc/ORC.java | 6 +++++- .../java/org/apache/iceberg/orc/OrcFileAppender.java | 10 ++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 1611eb49c7e9..5903ef70e8b4 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -19,6 +19,9 @@ package org.apache.iceberg.orc; +import static org.apache.iceberg.orc.OrcFileAppender.DEFAULT_BATCH_SIZE; +import static org.apache.iceberg.orc.OrcFileAppender.VECTOR_ROW_BATCH_SIZE; + import com.google.common.base.Preconditions; import java.nio.charset.StandardCharsets; import java.util.HashMap; @@ -88,7 +91,8 @@ public FileAppender build() { Preconditions.checkNotNull(schema, "Schema is required"); OrcFile.WriterOptions options = OrcFile.writerOptions(conf); return new OrcFileAppender<>(TypeConversion.toOrc(schema, new ColumnIdMap()), - this.file, createWriterFunc, options, metadata); + this.file, createWriterFunc, options, metadata, + conf.getInt(VECTOR_ROW_BATCH_SIZE, DEFAULT_BATCH_SIZE)); } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index 6c283493098e..6834e3f41ba8 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -41,6 +41,7 @@ * Create a file appender for ORC. */ class OrcFileAppender implements FileAppender { + private final int batchSize; private final TypeDescription orcSchema; private final ColumnIdMap columnIds = new ColumnIdMap(); private final Path path; @@ -49,14 +50,19 @@ class OrcFileAppender implements FileAppender { private final OrcValueWriter valueWriter; private boolean isClosed = false; + static final String VECTOR_ROW_BATCH_SIZE = "iceberg.orc.vectorbatch.size"; static final String COLUMN_NUMBERS_ATTRIBUTE = "iceberg.column.ids"; + static final int DEFAULT_BATCH_SIZE = DEFAULT_SIZE; + OrcFileAppender(TypeDescription schema, OutputFile file, Function> createWriterFunc, - OrcFile.WriterOptions options, Map metadata) { + OrcFile.WriterOptions options, Map metadata, + int batchSize) { orcSchema = schema; path = new Path(file.location()); - batch = orcSchema.createRowBatch(DEFAULT_SIZE); + this.batchSize = batchSize; + batch = orcSchema.createRowBatch(batchSize); options.setSchema(orcSchema); writer = newOrcWriter(file, columnIds, options, metadata); From c0429f01c5b0be04f679eea8f7f2664437455b33 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Mon, 6 May 2019 18:06:27 -0700 Subject: [PATCH 19/23] Update to ORC storage api 1.5.5 --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 0e6464109c76..1f677e0b97ea 100644 --- a/build.gradle +++ b/build.gradle @@ -76,7 +76,7 @@ subprojects { ext { hadoopVersion = '2.7.3' avroVersion = '1.8.2' - orcVersion = '1.5.0' + orcVersion = '1.5.5' parquetVersion = '1.10.0' hiveVersion = '1.2.1' From 29403e0f41ac23ef38de192fdb58190e9d224501 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Fri, 10 May 2019 16:56:51 -0700 Subject: [PATCH 20/23] Use VectorizedRowBatch for interface OrcValueReader --- .../main/java/org/apache/iceberg/orc/OrcValueReader.java | 4 +++- .../java/org/apache/iceberg/spark/data/SparkOrcReader.java | 7 +------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java index 0b48b3ad6598..cfc9ebb8afc3 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java @@ -19,6 +19,8 @@ package org.apache.iceberg.orc; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + /** * Used for implementing ORC value readers. */ @@ -27,6 +29,6 @@ public interface OrcValueReader { /** * Reads a value in row. */ - T read(Object reuse, int row); + T read(VectorizedRowBatch batch, int row); } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index dc5530a53360..a3de9c3ff174 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -75,12 +75,7 @@ private Converter[] buildConverters(final UnsafeRowWriter writer) { } @Override - public InternalRow read(Object reuse, int row) { - if (!(reuse instanceof VectorizedRowBatch)) { - throw new IllegalArgumentException("Value to read must be of type "+ VectorizedRowBatch.class); - } - - final VectorizedRowBatch batch = (VectorizedRowBatch) reuse; + public InternalRow read(VectorizedRowBatch batch, int row) { final UnsafeRowWriter rowWriter = new UnsafeRowWriter(numFields, INITIAL_SIZE); final Converter[] converters = buildConverters(rowWriter); From d5ae94ac52b02dcd53fdf385db161e7fdeb164d4 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Fri, 10 May 2019 17:03:22 -0700 Subject: [PATCH 21/23] Reset Writer to original imports --- .../org/apache/iceberg/spark/source/Writer.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java index 87e41605e97b..41241369a789 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -24,6 +24,14 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.function.Function; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -51,14 +59,6 @@ import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.function.Function; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.transform; From 4e785a9b99a7bee967fa03374ba9e229fa7bb23c Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Fri, 10 May 2019 18:07:29 -0700 Subject: [PATCH 22/23] Use ORC DEFAULT_SIZE for batch size --- orc/src/main/java/org/apache/iceberg/orc/ORC.java | 10 ++++++---- .../java/org/apache/iceberg/orc/OrcFileAppender.java | 11 +++-------- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 5903ef70e8b4..0e34f26c81f8 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -19,9 +19,6 @@ package org.apache.iceberg.orc; -import static org.apache.iceberg.orc.OrcFileAppender.DEFAULT_BATCH_SIZE; -import static org.apache.iceberg.orc.OrcFileAppender.VECTOR_ROW_BATCH_SIZE; - import com.google.common.base.Preconditions; import java.nio.charset.StandardCharsets; import java.util.HashMap; @@ -38,7 +35,12 @@ import org.apache.orc.OrcFile; import org.apache.orc.TypeDescription; +import static org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch.DEFAULT_SIZE; + public class ORC { + + static final String VECTOR_ROW_BATCH_SIZE = "iceberg.orc.vectorbatch.size"; + private ORC() { } @@ -92,7 +94,7 @@ public FileAppender build() { OrcFile.WriterOptions options = OrcFile.writerOptions(conf); return new OrcFileAppender<>(TypeConversion.toOrc(schema, new ColumnIdMap()), this.file, createWriterFunc, options, metadata, - conf.getInt(VECTOR_ROW_BATCH_SIZE, DEFAULT_BATCH_SIZE)); + conf.getInt(VECTOR_ROW_BATCH_SIZE, DEFAULT_SIZE)); } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index 6834e3f41ba8..60c738cd0678 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -35,8 +35,6 @@ import org.apache.orc.Writer; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; -import static org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch.DEFAULT_SIZE; - /** * Create a file appender for ORC. */ @@ -50,10 +48,7 @@ class OrcFileAppender implements FileAppender { private final OrcValueWriter valueWriter; private boolean isClosed = false; - static final String VECTOR_ROW_BATCH_SIZE = "iceberg.orc.vectorbatch.size"; - static final String COLUMN_NUMBERS_ATTRIBUTE = "iceberg.column.ids"; - - static final int DEFAULT_BATCH_SIZE = DEFAULT_SIZE; + private static final String COLUMN_NUMBERS_ATTRIBUTE = "iceberg.column.ids"; OrcFileAppender(TypeDescription schema, OutputFile file, Function> createWriterFunc, @@ -62,7 +57,7 @@ class OrcFileAppender implements FileAppender { orcSchema = schema; path = new Path(file.location()); this.batchSize = batchSize; - batch = orcSchema.createRowBatch(batchSize); + batch = orcSchema.createRowBatch(this.batchSize); options.setSchema(orcSchema); writer = newOrcWriter(file, columnIds, options, metadata); @@ -73,7 +68,7 @@ class OrcFileAppender implements FileAppender { public void add(D datum) { try { valueWriter.write(datum, batch); - if (batch.size == DEFAULT_SIZE) { + if (batch.size == this.batchSize) { writer.addRowBatch(batch); batch.reset(); } From bb0ef4069259c71c227fdafc291000360b32d248 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Thu, 9 May 2019 15:21:00 -0700 Subject: [PATCH 23/23] Propagate ORC case sensitive setting for read schema Rename orcSchema to readSchema since that's effectively the function it accomplishes. --- orc/src/main/java/org/apache/iceberg/orc/ORC.java | 6 ++++++ .../main/java/org/apache/iceberg/orc/OrcIterable.java | 7 ++++--- .../org/apache/iceberg/spark/data/SparkOrcReader.java | 9 ++++++--- .../java/org/apache/iceberg/spark/source/Reader.java | 1 + 4 files changed, 17 insertions(+), 6 deletions(-) diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 0e34f26c81f8..c6cb03649733 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -32,6 +32,7 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.orc.OrcConf; import org.apache.orc.OrcFile; import org.apache.orc.TypeDescription; @@ -139,6 +140,11 @@ public ReadBuilder schema(org.apache.iceberg.Schema schema) { return this; } + public ReadBuilder caseSensitive(boolean caseSensitive) { + OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(this.conf, caseSensitive); + return this; + } + public ReadBuilder config(String property, String value) { conf.set(property, value); return this; diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java index aeda7b5cd72e..b4bed83dad80 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java @@ -65,17 +65,18 @@ start, length, newFileReader(file, config)), readerFunction.apply(schema)); } - private static VectorizedRowBatchIterator newOrcIterator(InputFile file, TypeDescription orcSchema, + private static VectorizedRowBatchIterator newOrcIterator(InputFile file, + TypeDescription readerSchema, Long start, Long length, Reader orcFileReader) { final Reader.Options options = orcFileReader.options(); if (start != null) { options.range(start, length); } - options.schema(orcSchema); + options.schema(readerSchema); try { - return new VectorizedRowBatchIterator(file.location(), orcSchema, orcFileReader.rows(options)); + return new VectorizedRowBatchIterator(file.location(), readerSchema, orcFileReader.rows(options)); } catch (IOException ioe) { throw new RuntimeIOException(ioe, "Failed to get ORC rows for file: %s", file); } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index a3de9c3ff174..301a55064ae0 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -59,17 +59,17 @@ public class SparkOrcReader implements OrcValueReader { private final static int INITIAL_SIZE = 128 * 1024; private final int numFields; - private final TypeDescription orcSchema; + private final TypeDescription readSchema; public SparkOrcReader(Schema readSchema) { - orcSchema = TypeConversion.toOrc(readSchema, new ColumnIdMap()); + this.readSchema = TypeConversion.toOrc(readSchema, new ColumnIdMap()); numFields = readSchema.columns().size(); } private Converter[] buildConverters(final UnsafeRowWriter writer) { final Converter[] converters = new Converter[numFields]; for(int c = 0; c < numFields; ++c) { - converters[c] = buildConverter(writer, orcSchema.getChildren().get(c)); + converters[c] = buildConverter(writer, readSchema.getChildren().get(c)); } return converters; } @@ -96,6 +96,9 @@ private static String rowToString(SpecializedGetters row, TypeDescription schema rowBuilder.append(schema.getFieldNames().get(c)); rowBuilder.append("\": "); rowBuilder.append(rowEntryToString(row, c, children.get(c))); + if (c != children.size() - 1) { + rowBuilder.append(", "); + } } rowBuilder.append("}"); return rowBuilder.toString(); diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index 5772c6083ca4..63a33f958000 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -479,6 +479,7 @@ private CloseableIterable newOrcIterable(InputFile location, .schema(readSchema) .split(task.start(), task.length()) .createReaderFunc(SparkOrcReader::new) + .caseSensitive(caseSensitive) .build(); } }