You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by dongjoon-hyun <gi...@git.apache.org> on 2017/12/11 22:52:53 UTC

[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

GitHub user dongjoon-hyun opened a pull request:

    https://github.com/apache/spark/pull/19943

    [SPARK-16060][SQL] Support Vectorized ORC Reader

    ## What changes were proposed in this pull request?
    
    This PR adds a ORC columnar-batch reader to native `OrcFileFormat`.
    
    ## How was this patch tested?
    
    Pass the existing test cases.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/dongjoon-hyun/spark SPARK-16060

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19943.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19943
    
----
commit 449a0a2f8f58b28bc93837645ee542b555f4e6c3
Author: Dongjoon Hyun <do...@apache.org>
Date:   2017-12-06T08:12:02Z

    [SPARK-16060][SQL] Support Vectorized ORC Reader

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    After minimizing `nextBatch`, it becomes smaller than Parquet's `nextBatch`. But, it's inlined only some cases, but mostly not. It's not helpful. For the other technique, I'll try later.
    ```
    org.apache.spark.sql.execution.datasources.orc.OrcColumnarBatchReader::nextBatch (107 bytes)   inline (hot)
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84733/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160088124
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredSchema.length() == requestedColIds.length);
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.putNulls(0, batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              for (int index = 0; index < batchSize; index++) {
    +                toColumn.putByteArray(index, data.vector[0], data.start[0], data.length[0]);
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              putDecimalWritables(
    +                toColumn,
    +                batchSize,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putBoolean(index, data[index] == 1);
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByte(index, (byte)data[index]);
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putShort(index, (short)data[index]);
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putInt(index, (int)data[index]);
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putFloat(index, (float)data[index]);
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              putDecimalWritable(
    +                toColumn,
    +                index,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                data.vector[index]);
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        } else {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putBoolean(index, vector[index] == 1);
    +              }
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putByte(index, (byte)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putShort(index, (short)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putInt(index, (int)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof LongType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putLong(index, vector[index]);
    +              }
    +            }
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +              }
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putFloat(index, (float)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof DoubleType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putDouble(index, vector[index]);
    +              }
    +            }
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putByteArray(
    +                  index, vector.vector[index], vector.start[index], vector.length[index]);
    +              }
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                putDecimalWritable(
    +                  toColumn,
    +                  index,
    +                  decimalType.precision(),
    +                  decimalType.scale(),
    +                  vector[index]);
    +              }
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    +   * because they have different default values like the following.
    +   *
    +   * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024
    +   * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024
    +   */
    +  public static final int DEFAULT_SIZE = 4 * 1024;
    +
    +  /**
    +   * Returns the number of micros since epoch from an element of TimestampColumnVector.
    +   */
    +  private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) {
    +    return vector.time[index] * 1000L + vector.nanos[index] / 1000L;
    +  }
    +
    +  /**
    +   * Put a `HiveDecimalWritable` to a `WritableColumnVector`.
    +   */
    +  private static void putDecimalWritable(
    +      WritableColumnVector toColumn,
    +      int index,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +      Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInt(index, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLong(index, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      toColumn.putByteArray(index, bytes, 0, bytes.length);
    +    }
    +  }
    +
    +  /**
    +   * Put `HiveDecimalWritable`s to a `WritableColumnVector`.
    +   */
    +  private static void putDecimalWritables(
    +      WritableColumnVector toColumn,
    +      int size,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +            Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    --- End diff --
    
    2 space indentation


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85466/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160082910
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    --- End diff --
    
    Initially, both are 0.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85829 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85829/testReport)** for PR 19943 at commit [`db02555`](https://github.com/apache/spark/commit/db025552700f174686ddea9f6ea6f13078a64079).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    I answered at the comment~


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160360721
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala ---
    @@ -118,6 +118,13 @@ class OrcFileFormat
         }
       }
     
    +  override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = {
    +    val conf = sparkSession.sessionState.conf
    +    conf.orcVectorizedReaderEnabled && conf.wholeStageEnabled &&
    +      schema.length <= conf.wholeStageMaxNumFields &&
    +      schema.forall(_.dataType.isInstanceOf[AtomicType])
    +  }
    +
    --- End diff --
    
    Do we need to implement `vectorTypes` as `ParquetFileFormat`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069510
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    --- End diff --
    
    nit: For this kind of simple one line comment, use `// xxx`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156470105
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    --- End diff --
    
    Yep. I'll add assertion.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158769195
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    --- End diff --
    
    Should we add an `assert` to make sure `batch` is initialized (not null)? 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160182013
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    --- End diff --
    
    will we output less data? Assume one batch has 0 rows after PPD, but the ORC file has more batches to return, what should we do?
    
    Also we may report wrong progress, since the `totalRowCount` is the number of rows in the ORC file, while `rowsReturned` is the number of rows returned after PPD.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by henrify <gi...@git.apache.org>.
Github user henrify commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @cloud-fan Oh you are right, it is indeed byte[][]. The BytesColumnVector has separate per-row offset and length vectors/arrays, which seemed to indicate that it would be contiguous block, and those auxiliary arrays/vectors are used to mark the boundaries. I'm not sure why they need those two additional vectors then.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by henrify <gi...@git.apache.org>.
Github user henrify commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160084073
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]);
    +                index += 1;
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendBoolean(data[index] == 1);
    --- End diff --
    
    Thinking this further, the ColumnVector API supports direct memory copy. For long, double, binary and string you might be able to scrap the for loops entirely, and copy ORC's memory directly to Spark's memory, with just 1 call per batch, which would be way faster than looping. Check e.g. putDoubles(int rowId, int count, double[] src, int srcIndex).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85792 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85792/testReport)** for PR 19943 at commit [`3df7d1e`](https://github.com/apache/spark/commit/3df7d1ee9d0cb9ea25a9d1e0e2db539121ad50de).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158894676
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    --- End diff --
    
    Yep. It's renamed to `recordReader`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160016468
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          156 /  163        100.7           9.9       1.0X
    +        Native ORC MR                                 1222 / 1236         12.9          77.7       0.1X
    +        Hive built-in ORC                             1572 / 1625         10.0         100.0       0.1X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          215 /  225         73.1          13.7       1.0X
    +        Native ORC MR                                 1337 / 1458         11.8          85.0       0.2X
    +        Hive built-in ORC                             1696 / 1707          9.3         107.8       0.1X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          285 /  294         55.2          18.1       1.0X
    +        Native ORC MR                                 1397 / 1419         11.3          88.8       0.2X
    +        Hive built-in ORC                             2086 / 2143          7.5         132.6       0.1X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          341 /  350         46.1          21.7       1.0X
    +        Native ORC MR                                 1461 / 1492         10.8          92.9       0.2X
    +        Hive built-in ORC                             2002 / 2095          7.9         127.3       0.2X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          347 /  356         45.3          22.1       1.0X
    +        Native ORC MR                                 1524 / 1553         10.3          96.9       0.2X
    +        Hive built-in ORC                             2393 / 2404          6.6         152.1       0.1X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          403 /  411         39.0          25.6       1.0X
    +        Native ORC MR                                 1517 / 1529         10.4          96.5       0.3X
    +        Hive built-in ORC                             2054 / 2134          7.7         130.6       0.2X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                         1382 / 1400          7.6         131.8       1.0X
    +        Native ORC MR                                 2689 / 2765          3.9         256.4       0.5X
    +        Hive built-in ORC                             3889 / 3894          2.7         370.9       0.4X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC Vectorized       347 /  350         45.3          22.1       1.0X
    +        Read data column - Native ORC MR              1611 / 1613          9.8         102.4       0.2X
    +        Read data column - Hive built-in ORC          2082 / 2091          7.6         132.4       0.2X
    +        Read partition column - Native ORC Vectorized   55 /   57        286.5           3.5       6.3X
    +        Read partition column - Native ORC MR         1062 / 1063         14.8          67.5       0.3X
    +        Read partition column - Hive built-in ORC     1334 / 1334         11.8          84.8       0.3X
    +        Read both columns - Native ORC Vectorized      380 /  388         41.3          24.2       0.9X
    +        Read both columns - Native ORC MR             1654 / 1672          9.5         105.2       0.2X
    +        Read both columns - Hive built-in ORC         2209 / 2209          7.1         140.5       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          406 /  414         25.8          38.7       1.0X
    +        Native ORC MR                                 1372 / 1381          7.6         130.8       0.3X
    +        Hive built-in ORC                             2016 / 2036          5.2         192.2       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized (0.0%)                  1122 / 1129          9.3         107.0       1.0X
    +        Native ORC MR (0.0%)                          2551 / 2619          4.1         243.3       0.4X
    --- End diff --
    
    switch the order so that we can use `Native ORC MR` as the baseline, then the `Relative` part can be more readable.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158447746
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    --- End diff --
    
    do we need to close the orc batch?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158895185
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    --- End diff --
    
    `The column ids not requested are fixed` is done by OrcDeserializer.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158894279
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    Sorry?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    A high-level question: @viirya had a PR to do this by creating a wrapper for ORC columnar batch. The parquet data source pick a different approach that writes the values to Spark columnar batch.
    
    Generally I think the wrapper approach should be faster on pure scan, but may be slower if there is computation after the scan, e.g. aggregate. Do we have a benchmark for it?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85818/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069857
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    --- End diff --
    
    `== -1` is more precise


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85674 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85674/testReport)** for PR 19943 at commit [`83cc3b5`](https://github.com/apache/spark/commit/83cc3b5b327faa40ebd114f9c7d8f38326a30b5b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85823/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85534 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85534/testReport)** for PR 19943 at commit [`9f225e2`](https://github.com/apache/spark/commit/9f225e23b562c5baa7d137b13c697cdeca5e9d43).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85792 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85792/testReport)** for PR 19943 at commit [`3df7d1e`](https://github.com/apache/spark/commit/3df7d1ee9d0cb9ea25a9d1e0e2db539121ad50de).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85793 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85793/testReport)** for PR 19943 at commit [`15cac9c`](https://github.com/apache/spark/commit/15cac9cf6b99415b03fc818fbb14a16b722c9058).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158895355
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,357 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.Random
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     132 /  138        119.4           8.4       1.0X
    +        Hive built-in ORC                             1328 / 1333         11.8          84.5       0.1X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     178 /  188         88.2          11.3       1.0X
    +        Hive built-in ORC                             1541 / 1560         10.2          98.0       0.1X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     242 /  256         64.9          15.4       1.0X
    +        Hive built-in ORC                             1650 / 1676          9.5         104.9       0.1X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     297 /  309         53.0          18.9       1.0X
    +        Hive built-in ORC                             1750 / 1766          9.0         111.3       0.2X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     352 /  363         44.7          22.4       1.0X
    +        Hive built-in ORC                             1749 / 1764          9.0         111.2       0.2X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     436 /  456         36.1          27.7       1.0X
    +        Hive built-in ORC                             1852 / 1860          8.5         117.8       0.2X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                    1374 / 1376          7.6         131.0       1.0X
    +        Hive built-in ORC                             3653 / 3664          2.9         348.4       0.4X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC                  321 /  327         49.0          20.4       1.0X
    +        Read data column - Hive built-in ORC          2041 / 2176          7.7         129.8       0.2X
    +        Read partition column - Native ORC              53 /   57        298.2           3.4       6.1X
    +        Read partition column - Hive built-in ORC     1176 / 1183         13.4          74.7       0.3X
    +        Read both columns - Native ORC                 335 /  340         47.0          21.3       1.0X
    +        Read both columns - Hive built-in ORC         1970 / 1974          8.0         125.2       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     363 /  382         28.9          34.7       1.0X
    +        Hive built-in ORC                             2012 / 2080          5.2         191.9       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC ($fractionOfNulls%)") { iter =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { iter =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC (0.0%)                             1120 / 1142          9.4         106.8       1.0X
    +        Hive built-in ORC (0.0%)                      4232 / 4284          2.5         403.6       0.3X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC (0.5%)                             1474 / 1512          7.1         140.5       1.0X
    +        Hive built-in ORC (0.5%)                      3114 / 3140          3.4         297.0       0.5X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC (0.95%)                             568 /  589         18.5          54.1       1.0X
    +        Hive built-in ORC (0.95%)                     1548 / 1549          6.8         147.6       0.4X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan FROM $width-Column Rows", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT * FROM t1"))
    --- End diff --
    
    Thanks. It's removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158895273
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    --- End diff --
    
    Yep. I merged it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85829 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85829/testReport)** for PR 19943 at commit [`db02555`](https://github.com/apache/spark/commit/db025552700f174686ddea9f6ea6f13078a64079).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158818091
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    --- End diff --
    
    Can this be merged with `IntegerType` too?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #84788 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84788/testReport)** for PR 19943 at commit [`d302259`](https://github.com/apache/spark/commit/d30225993c2c674898ea43e64b555b1283ca8e0c).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160178331
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    --- End diff --
    
    yup


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160093941
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredSchema.length() == requestedColIds.length);
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.putNulls(0, batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    --- End diff --
    
    Thanks. I updated like the following.
    ```
    toColumn.putByteArray(0, data.vector[0]);
    for (int index = 0; index < batchSize; index++) {
      toColumn.putArray(index, data.start[0], data.length[0]);
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160311799
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,528 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +import java.util.stream.IntStream;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +        int size = data.vector[0].length;
    +        arrayData.reserve(size);
    +        arrayData.putBytes(0, size, data.vector[0], 0);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, 0, size);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      int totalNumBytes = IntStream.of(data.length).sum();
    +      arrayData.reserve(totalNumBytes);
    +      for (int index = 0, pos = 0; index < batchSize; pos += data.length[index], index++) {
    +        arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]);
    +        toColumn.putArray(index, pos, data.length[index]);
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    +      DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +      if (decimalType.precision() > Decimal.MAX_INT_DIGITS()) {
    +        WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +        arrayData.reserve(batchSize * 16);
    +      }
    +      for (int index = 0; index < batchSize; index++) {
    +        putDecimalWritable(
    +          toColumn,
    +          index,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          data.vector[index]);
    +      }
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +  private void putValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putBoolean(index, vector[index] == 1);
    +        }
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putByte(index, (byte)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putShort(index, (short)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putInt(index, (int)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof LongType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +        }
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putFloat(index, (float)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof DoubleType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putDouble(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      int totalNumBytes = IntStream.of(vector.length).sum();
    +      arrayData.reserve(totalNumBytes);
    +      for (int index = 0, pos = 0; index < batchSize; pos += vector.length[index], index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          arrayData.putBytes(pos, vector.length[index], vector.vector[index], vector.start[index]);
    +          toColumn.putArray(index, pos, vector.length[index]);
    +        }
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    +      HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    --- End diff --
    
    Oh, I see. It's the following. Thanks!
    ```
    if (type.precision() > Decimal.MAX_LONG_DIGITS()) {
    ...
    ```



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Thank you so much, @cloud-fan , @mmccline , @viirya , @henrify , @kiszk , @HyukjinKwon !
    I'll proceed to follow-ups.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85744 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85744/testReport)** for PR 19943 at commit [`aeb6abd`](https://github.com/apache/spark/commit/aeb6abd66ee3338635edf9dca85894c14a05fb72).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156468994
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L)
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale)
    +                value.changePrecision(precision, scale)
    +                if (precision <= Decimal.MAX_INT_DIGITS) {
    +                  toColumn.appendInts(batchSize, value.toUnscaledLong.toInt)
    +                } else if (precision <= Decimal.MAX_LONG_DIGITS) {
    +                  toColumn.appendLongs(batchSize, value.toUnscaledLong)
    +                } else {
    +                  val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray
    +                  var index = 0
    +                  while (index < batchSize) {
    +                    toColumn.appendByteArray(bytes, 0, bytes.length)
    +                    index += 1
    +                  }
    +                }
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    --- End diff --
    
    Could you elaborate your concern on this?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85534/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158899621
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    nvm


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160016431
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,493 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.vectorized._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +    totalRowCount = reader.getNumberOfRows
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    --- End diff --
    
    similar to `VectorizedParquetRecordReader.initBatch`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159627680
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala ---
    @@ -110,4 +107,22 @@ object OrcUtils extends Logging {
           }
         }
       }
    +
    +  /**
    +   * Return a fixed ORC schema with data schema information, if needed.
    +   * The schema inside old ORC files might consist of invalid column names like '_col0'.
    +   */
    +  def getFixedTypeDescription(
    +      schema: TypeDescription,
    +      dataSchema: StructType): TypeDescription = {
    +    if (schema.getFieldNames.asScala.forall(_.startsWith("_col"))) {
    +      var schemaString = schema.toString
    +      dataSchema.zipWithIndex.foreach { case (field: StructField, index: Int) =>
    --- End diff --
    
    Yep. I added the condition into `if`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159221380
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    --- End diff --
    
    shall we handle missing columns like the parquet reader does?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85772 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85772/testReport)** for PR 19943 at commit [`7214ec0`](https://github.com/apache/spark/commit/7214ec03f8e48d51e0fd1f3314a0af6ac8275412).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> `


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85674/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156241635
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala ---
    @@ -139,15 +146,25 @@ class OrcFileFormat
           }
         }
     
    +    val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
    +    val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled &&
    +      supportBatch(sparkSession, resultSchema)
    +
         val broadcastedConf =
           sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
         val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
     
         (file: PartitionedFile) => {
           val conf = broadcastedConf.value.value
     
    +      val filePath = new Path(new URI(file.filePath))
    +
    +      val fs = filePath.getFileSystem(conf)
    +      val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
    +      val reader = OrcFile.createReader(filePath, readerOptions)
    --- End diff --
    
    Why extract the creation of `reader` from `requestedColumnIds` to here?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158764573
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByte(data(index).toByte)
    +                index += 1
    +              }
    +            case ShortType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    ditto


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Thank you for your help, @henrify . I think it's within margin of deviation.
    Split methods will be better for maintenance. So, I pushed it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158767130
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    --- End diff --
    
    +1


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159116359
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    --- End diff --
    
    I meant, the columns not requested are not changed for each batch. For now every time you reset the batch and then re-append nulls into such columns. But those columns can be reused.
    
    Once #20116 is merged, we reset by column vectors, not the columnar batch. So we can just reset the column vectors of the required columns.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160016423
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,493 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.vectorized._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +    totalRowCount = reader.getNumberOfRows
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    --- End diff --
    
    I feel it's more intuitive to pass in `requiredSchema` and `partitionSchema`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160017124
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,493 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.vectorized._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +    totalRowCount = reader.getNumberOfRows
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse, "No filters are allowed")
    --- End diff --
    
    No, ORC fills the batch after applying the push-downed filters.
    `selectedInUse` is used by some operators of Hive in order to generate derived row-batches. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    That's a good point, @cloud-fan . I also agreed to see the result for that before (at the my initial PR.)
    
    First of all, we can compare the results with `OrcReadBenchmark.scala` in this PR before going with aggregation. Since we have the foundation for that, @viirya can update his PR more easily than before. I love to see the fastest Apache Spark ORC!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by mmccline <gi...@git.apache.org>.
Github user mmccline commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156229525
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L)
    --- End diff --
    
    This raw calculation here should be encapsulated in a utilities method with a few comments explaining what is happening (i.e. the conversion from TimestampColumnVector's java.sql.Timestamp representation to Spark's).  It is repeated (so to speak several times in this method/


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160133388
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,517 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1161 / 1168         13.5          73.8       1.0X
    +        Native ORC Vectorized                          163 /  171         96.3          10.4       7.1X
    +        Native ORC Vectorized (Java)                   155 /  163        101.6           9.8       7.5X
    +        Hive built-in ORC                             1427 / 1427         11.0          90.7       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1261 / 1321         12.5          80.2       1.0X
    +        Native ORC Vectorized                          160 /  167         98.2          10.2       7.9X
    +        Native ORC Vectorized (Java)                   160 /  167         98.4          10.2       7.9X
    +        Hive built-in ORC                             1655 / 1687          9.5         105.2       0.8X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1369 / 1449         11.5          87.1       1.0X
    +        Native ORC Vectorized                          263 /  277         59.8          16.7       5.2X
    +        Native ORC Vectorized (Java)                   225 /  237         70.0          14.3       6.1X
    +        Hive built-in ORC                             1867 / 1899          8.4         118.7       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1472 / 1474         10.7          93.6       1.0X
    +        Native ORC Vectorized                          289 /  300         54.5          18.4       5.1X
    +        Native ORC Vectorized (Java)                   286 /  294         54.9          18.2       5.1X
    +        Hive built-in ORC                             1917 / 1934          8.2         121.9       0.8X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1484 / 1484         10.6          94.3       1.0X
    +        Native ORC Vectorized                          365 /  370         43.1          23.2       4.1X
    +        Native ORC Vectorized (Java)                   326 /  335         48.2          20.7       4.5X
    +        Hive built-in ORC                             1978 / 2049          8.0         125.8       0.8X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1550 / 1554         10.1          98.6       1.0X
    +        Native ORC Vectorized                          396 /  405         39.7          25.2       3.9X
    +        Native ORC Vectorized (Java)                   394 /  402         39.9          25.1       3.9X
    +        Hive built-in ORC                             2072 / 2084          7.6         131.8       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2636 / 2734          4.0         251.4       1.0X
    +        Native ORC Vectorized                         1267 / 1267          8.3         120.9       2.1X
    +        Native ORC Vectorized (Java)                  1182 / 1183          8.9         112.7       2.2X
    +        Hive built-in ORC                             3724 / 3764          2.8         355.2       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR                    1587 / 1592          9.9         100.9       1.0X
    +        Read data column - Native ORC Vectorized             290 /  309         54.3          18.4       5.5X
    +        Read data column - Native ORC Vectorized (Java)      293 /  297         53.7          18.6       5.4X
    +        Read data column - Hive built-in ORC                2204 / 2214          7.1         140.1       0.7X
    +        Read partition column - Native ORC MR               1078 / 1097         14.6          68.5       1.5X
    +        Read partition column - Native ORC Vectorized         53 /   56        294.0           3.4      29.7X
    +        Read partition column - Native ORC Vectorized (Java)  52 /   55        300.7           3.3      30.4X
    +        Read partition column - Hive built-in ORC           1279 / 1287         12.3          81.3       1.2X
    +        Read both columns - Native ORC MR                   1665 / 1674          9.4         105.9       1.0X
    +        Read both columns - Native ORC Vectorized            327 /  333         48.0          20.8       4.8X
    +        Read both columns - Native ORC Vectorized (Java)     327 /  332         48.2          20.8       4.9X
    +        Read both columns - Hive built-in ORC               2157 / 2169          7.3         137.1       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1355 / 1355          7.7         129.2       1.0X
    +        Native ORC Vectorized                          262 /  270         40.0          25.0       5.2X
    +        Native ORC Vectorized (Java)                   223 /  227         46.9          21.3       6.1X
    +        Hive built-in ORC                             2017 / 2027          5.2         192.4       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%) (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.0%)                          2575 / 2618          4.1         245.5       1.0X
    +        Native ORC Vectorized (0.0%)                   841 /  852         12.5          80.2       3.1X
    +        Native ORC Vectorized (0.0%) (Java)            757 /  760         13.9          72.2       3.4X
    +        Hive built-in ORC (0.0%)                      4149 / 4162          2.5         395.7       0.6X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.5%)                          2446 / 2460          4.3         233.3       1.0X
    +        Native ORC Vectorized (0.5%)                  1081 / 1084          9.7         103.1       2.3X
    +        Native ORC Vectorized (0.5%) (Java)           1066 / 1069          9.8         101.6       2.3X
    +        Hive built-in ORC (0.5%)                      2928 / 2938          3.6         279.2       0.8X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.95%)                         1287 / 1331          8.2         122.7       1.0X
    +        Native ORC Vectorized (0.95%)                  404 /  407         26.0          38.5       3.2X
    +        Native ORC Vectorized (0.95%) (Java)           405 /  409         25.9          38.6       3.2X
    +        Hive built-in ORC (0.95%)                     1612 / 1644          6.5         153.7       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from wide table ($width columns)", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1116 / 1117          0.9        1064.7       1.0X
    +        Native ORC Vectorized                           92 /   98         11.3          88.2      12.1X
    +        Native ORC Vectorized (Java)                    90 /   96         11.6          86.0      12.4X
    +        Hive built-in ORC                              376 /  386          2.8         358.6       3.0X
    --- End diff --
    
    Ur, I meant only current ORC readers. I didn't check the detail of old Hive ORC reader.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156468718
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L)
    --- End diff --
    
    It's done.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160077928
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    I'm still testing some other stuff this PR.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160089616
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078039
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    --- End diff --
    
    Yep. It's removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160088987
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredSchema.length() == requestedColIds.length);
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.putNulls(0, batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              for (int index = 0; index < batchSize; index++) {
    +                toColumn.putByteArray(index, data.vector[0], data.start[0], data.length[0]);
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              putDecimalWritables(
    +                toColumn,
    +                batchSize,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putBoolean(index, data[index] == 1);
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByte(index, (byte)data[index]);
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putShort(index, (short)data[index]);
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putInt(index, (int)data[index]);
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putFloat(index, (float)data[index]);
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +            }
    +          } else if (type instanceof DecimalType) {
    --- End diff --
    
    We missed string/binary here.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @cloud-fan . According to your advice, I added JavaOrcColumnarBatchReader and compared the result. Could you review the PR again?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160079119
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]);
    +                index += 1;
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendBoolean(data[index] == 1);
    +              index += 1;
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendByte((byte)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendShort((short)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendInt((int)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendLong(fromTimestampColumnVector(data, index));
    +              index += 1;
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendFloat((float)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendByteArray(data.vector[index], data.start[index], data.length[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                data.vector[index]);
    +              index += 1;
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        } else {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendBoolean(vector[index] == 1);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendByte((byte)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendShort((short)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendInt((int)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof LongType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendLong(vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendLong(fromTimestampColumnVector(vector, index));
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendFloat((float)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof DoubleType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendDouble(vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendByteArray(
    +                  vector.vector[index], vector.start[index], vector.length[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                appendDecimalWritable(
    +                  toColumn,
    +                  decimalType.precision(),
    +                  decimalType.scale(),
    +                  vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        }
    +      }
    +      i += 1;
    +    }
    +    return true;
    +  }
    +
    +  /**
    +   * Default memory mode for ColumnarBatch.
    +   */
    +  public static final MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP;
    --- End diff --
    
    Then, we need the following, too.
    ```scala
            val vectorizedReader = new VectorizedParquetRecordReader(
              convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined)
    ```
    
    ```scala
      /**
       * The memory mode of the columnarBatch
       */
      private final MemoryMode MEMORY_MODE;
    
      public VectorizedParquetRecordReader(TimeZone convertTz, boolean useOffHeap) {
        this.convertTz = convertTz;
        MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
      }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by henrify <gi...@git.apache.org>.
Github user henrify commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @dongjoon-hyun Thanks. I don't think it matters if nextBatch() is inlined or not. I think what matters is 1) how the putX() etc methods calls inside the tight loops are inlined and 2) how complex the methods containing the tight loops are.
    
    For example the toColumn argument is megamorphic and the putX() implementation is bimorphic, and then you have about 10 of these in single method inside if-else 'instanceof' checks. That's quite complex for JVM to optimize.
    
    If you split the loops so that each loop has it's own method with the toColumn defined as exact type (BytesColumnVector etc), then the argument is monomorphic, putX() is 100% biased bimorphic, and there is only one of these. Lot easier for JVM to optimize.
    
    Again, i'm not sure if it makes difference, but it may, and it is easy to try (e.g. extract the for loops of just one data type to separate method and benchmark).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85745 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85745/testReport)** for PR 19943 at commit [`aeb6abd`](https://github.com/apache/spark/commit/aeb6abd66ee3338635edf9dca85894c14a05fb72).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Thank you so much, @HyukjinKwon .


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159626907
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    --- End diff --
    
    I see, @viirya .


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by henrify <gi...@git.apache.org>.
Github user henrify commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160318348
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,605 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +import java.util.stream.IntStream;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    +   * because they have different default values like the following.
    +   *
    +   * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024
    +   * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024
    +   */
    +  public static final int DEFAULT_SIZE = 4 * 1024;
    +
    +  /**
    +   * Returns the number of micros since epoch from an element of TimestampColumnVector.
    +   */
    +  private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) {
    +    return vector.time[index] * 1000L + vector.nanos[index] / 1000L;
    +  }
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return recordReader.getProgress();
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        putByteArrays(batchSize, toColumn, ((BytesColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      putNonNullBooleans(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof ByteType) {
    +      putNonNullBytes(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof ShortType) {
    +      putNonNullShorts(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      putNonNullInts(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      putNonNullTimestamps(batchSize, (TimestampColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof FloatType) {
    +      putNonNullFloats(batchSize, (DoubleColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      putNonNullByteArray(batchSize, (BytesColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof DecimalType) {
    +      putNonNullDecimals(batchSize, (DecimalColumnVector) fromColumn, toColumn, (DecimalType) type);
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +
    +  private void putValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      putBooleans(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof ByteType) {
    +      putBytes(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof ShortType) {
    +      putShorts(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      putInts(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof LongType) {
    +      putLongs(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof TimestampType) {
    +      putTimestamps(batchSize, (TimestampColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof FloatType) {
    +      putFloats(batchSize, (DoubleColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof DoubleType) {
    +      putDoubles(batchSize, (DoubleColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      putByteArrays(batchSize, (BytesColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof DecimalType) {
    +      putDecimals(batchSize, (DecimalColumnVector)fromColumn, toColumn, (DecimalType) type);
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +  // --------------------------------------------------------------------------
    +  // Put a value
    +  // --------------------------------------------------------------------------
    +
    +  private static void putDecimalWritable(
    +      WritableColumnVector toColumn,
    +      int index,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +      Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInt(index, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLong(index, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      arrayData.putBytes(index * 16, bytes.length, bytes, 0);
    +      toColumn.putArray(index, index * 16, bytes.length);
    +    }
    +  }
    +
    +  // --------------------------------------------------------------------------
    +  // Put repeating values
    +  // --------------------------------------------------------------------------
    +
    +  private void putByteArrays(int count, WritableColumnVector toColumn, byte[] bytes) {
    +    WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +    int size = bytes.length;
    +    arrayData.reserve(size);
    +    arrayData.putBytes(0, size, bytes, 0);
    +    for (int index = 0; index < count; index++) {
    +      toColumn.putArray(index, 0, size);
    +    }
    +  }
    +
    +  private static void putDecimalWritables(
    +      WritableColumnVector toColumn,
    +      int size,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +      Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInts(0, size, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLongs(0, size, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      arrayData.reserve(bytes.length);
    +      arrayData.putBytes(0, bytes.length, bytes, 0);
    +      for (int index = 0; index < size; index++) {
    +        toColumn.putArray(index, 0, bytes.length);
    +      }
    +    }
    +  }
    +
    +  // --------------------------------------------------------------------------
    +  // Put non-null values
    +  // --------------------------------------------------------------------------
    +
    +  private void putNonNullBooleans(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) {
    +    long[] data = fromColumn.vector;
    +    for (int index = 0; index < count; index++) {
    +      toColumn.putBoolean(index, data[index] == 1);
    +    }
    +  }
    +
    +  private void putNonNullByteArray(int count, BytesColumnVector fromColumn, WritableColumnVector toColumn) {
    +    BytesColumnVector data = fromColumn;
    +    WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +    int totalNumBytes = IntStream.of(data.length).sum();
    +    arrayData.reserve(totalNumBytes);
    +    for (int index = 0, pos = 0; index < count; pos += data.length[index], index++) {
    +      arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]);
    +      toColumn.putArray(index, pos, data.length[index]);
    +    }
    +  }
    +
    +  private void putNonNullBytes(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) {
    --- End diff --
    
    It almost seemed that they hurt the performance. The MR and Hive tests were up and down randomly as expected, but Vectorized tests were down in almost every benchmark.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160091457
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredSchema.length() == requestedColIds.length);
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.putNulls(0, batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              for (int index = 0; index < batchSize; index++) {
    +                toColumn.putByteArray(index, data.vector[0], data.start[0], data.length[0]);
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              putDecimalWritables(
    +                toColumn,
    +                batchSize,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putBoolean(index, data[index] == 1);
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByte(index, (byte)data[index]);
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putShort(index, (short)data[index]);
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putInt(index, (int)data[index]);
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putFloat(index, (float)data[index]);
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              putDecimalWritable(
    +                toColumn,
    +                index,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                data.vector[index]);
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        } else {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putBoolean(index, vector[index] == 1);
    +              }
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putByte(index, (byte)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putShort(index, (short)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putInt(index, (int)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof LongType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putLong(index, vector[index]);
    +              }
    +            }
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +              }
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putFloat(index, (float)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof DoubleType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putDouble(index, vector[index]);
    +              }
    +            }
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putByteArray(
    +                  index, vector.vector[index], vector.start[index], vector.length[index]);
    +              }
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                putDecimalWritable(
    +                  toColumn,
    +                  index,
    +                  decimalType.precision(),
    +                  decimalType.scale(),
    +                  vector[index]);
    +              }
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    +   * because they have different default values like the following.
    +   *
    +   * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024
    +   * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024
    +   */
    +  public static final int DEFAULT_SIZE = 4 * 1024;
    +
    +  /**
    +   * Returns the number of micros since epoch from an element of TimestampColumnVector.
    +   */
    +  private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) {
    +    return vector.time[index] * 1000L + vector.nanos[index] / 1000L;
    +  }
    +
    +  /**
    +   * Put a `HiveDecimalWritable` to a `WritableColumnVector`.
    +   */
    +  private static void putDecimalWritable(
    +      WritableColumnVector toColumn,
    +      int index,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +      Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInt(index, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLong(index, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      toColumn.putByteArray(index, bytes, 0, bytes.length);
    +    }
    +  }
    +
    +  /**
    +   * Put `HiveDecimalWritable`s to a `WritableColumnVector`.
    +   */
    +  private static void putDecimalWritables(
    +      WritableColumnVector toColumn,
    +      int size,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +            Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInts(0, size, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLongs(0, size, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      for (int index = 0; index < size; index++) {
    +        toColumn.putByteArray(index, bytes, 0, bytes.length);
    --- End diff --
    
    we can apply https://github.com/apache/spark/pull/19943/files/3a0702ae0b31f762c9f3da06d267a02ec8d1a23b#r160088927 to here too


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85794 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85794/testReport)** for PR 19943 at commit [`10e5d7a`](https://github.com/apache/spark/commit/10e5d7a4bbac748019508fe3104e48c392696d9f).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85794 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85794/testReport)** for PR 19943 at commit [`10e5d7a`](https://github.com/apache/spark/commit/10e5d7a4bbac748019508fe3104e48c392696d9f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160164794
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    +      DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        putDecimalWritable(
    +          toColumn,
    +          index,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          data.vector[index]);
    +      }
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +  private void putValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putBoolean(index, vector[index] == 1);
    +        }
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putByte(index, (byte)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putShort(index, (short)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putInt(index, (int)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof LongType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +        }
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putFloat(index, (float)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof DoubleType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putDouble(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector vector = (BytesColumnVector)fromColumn;
    --- End diff --
    
    ditto


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Thank you for review, @mmccline !


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85808/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @henrify  and @cloud-fan . I updated the PR with put APIs. You can check the BM result.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85792/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160017477
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala ---
    @@ -110,4 +107,23 @@ object OrcUtils extends Logging {
           }
         }
       }
    +
    +  /**
    +   * Return a fixed ORC schema with data schema information, if needed.
    +   * The schema inside old ORC files might consist of invalid column names like '_col0'.
    +   */
    +  def getFixedTypeDescription(
    --- End diff --
    
    Okay. I'll remove this from this PR.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85744/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160123835
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,517 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1161 / 1168         13.5          73.8       1.0X
    +        Native ORC Vectorized                          163 /  171         96.3          10.4       7.1X
    +        Native ORC Vectorized (Java)                   155 /  163        101.6           9.8       7.5X
    +        Hive built-in ORC                             1427 / 1427         11.0          90.7       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1261 / 1321         12.5          80.2       1.0X
    +        Native ORC Vectorized                          160 /  167         98.2          10.2       7.9X
    +        Native ORC Vectorized (Java)                   160 /  167         98.4          10.2       7.9X
    +        Hive built-in ORC                             1655 / 1687          9.5         105.2       0.8X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1369 / 1449         11.5          87.1       1.0X
    +        Native ORC Vectorized                          263 /  277         59.8          16.7       5.2X
    +        Native ORC Vectorized (Java)                   225 /  237         70.0          14.3       6.1X
    +        Hive built-in ORC                             1867 / 1899          8.4         118.7       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1472 / 1474         10.7          93.6       1.0X
    +        Native ORC Vectorized                          289 /  300         54.5          18.4       5.1X
    +        Native ORC Vectorized (Java)                   286 /  294         54.9          18.2       5.1X
    +        Hive built-in ORC                             1917 / 1934          8.2         121.9       0.8X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1484 / 1484         10.6          94.3       1.0X
    +        Native ORC Vectorized                          365 /  370         43.1          23.2       4.1X
    +        Native ORC Vectorized (Java)                   326 /  335         48.2          20.7       4.5X
    +        Hive built-in ORC                             1978 / 2049          8.0         125.8       0.8X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1550 / 1554         10.1          98.6       1.0X
    +        Native ORC Vectorized                          396 /  405         39.7          25.2       3.9X
    +        Native ORC Vectorized (Java)                   394 /  402         39.9          25.1       3.9X
    +        Hive built-in ORC                             2072 / 2084          7.6         131.8       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2636 / 2734          4.0         251.4       1.0X
    +        Native ORC Vectorized                         1267 / 1267          8.3         120.9       2.1X
    +        Native ORC Vectorized (Java)                  1182 / 1183          8.9         112.7       2.2X
    +        Hive built-in ORC                             3724 / 3764          2.8         355.2       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR                    1587 / 1592          9.9         100.9       1.0X
    +        Read data column - Native ORC Vectorized             290 /  309         54.3          18.4       5.5X
    +        Read data column - Native ORC Vectorized (Java)      293 /  297         53.7          18.6       5.4X
    +        Read data column - Hive built-in ORC                2204 / 2214          7.1         140.1       0.7X
    +        Read partition column - Native ORC MR               1078 / 1097         14.6          68.5       1.5X
    +        Read partition column - Native ORC Vectorized         53 /   56        294.0           3.4      29.7X
    +        Read partition column - Native ORC Vectorized (Java)  52 /   55        300.7           3.3      30.4X
    +        Read partition column - Hive built-in ORC           1279 / 1287         12.3          81.3       1.2X
    +        Read both columns - Native ORC MR                   1665 / 1674          9.4         105.9       1.0X
    +        Read both columns - Native ORC Vectorized            327 /  333         48.0          20.8       4.8X
    +        Read both columns - Native ORC Vectorized (Java)     327 /  332         48.2          20.8       4.9X
    +        Read both columns - Hive built-in ORC               2157 / 2169          7.3         137.1       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1355 / 1355          7.7         129.2       1.0X
    +        Native ORC Vectorized                          262 /  270         40.0          25.0       5.2X
    +        Native ORC Vectorized (Java)                   223 /  227         46.9          21.3       6.1X
    +        Hive built-in ORC                             2017 / 2027          5.2         192.4       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%) (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.0%)                          2575 / 2618          4.1         245.5       1.0X
    +        Native ORC Vectorized (0.0%)                   841 /  852         12.5          80.2       3.1X
    +        Native ORC Vectorized (0.0%) (Java)            757 /  760         13.9          72.2       3.4X
    +        Hive built-in ORC (0.0%)                      4149 / 4162          2.5         395.7       0.6X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.5%)                          2446 / 2460          4.3         233.3       1.0X
    +        Native ORC Vectorized (0.5%)                  1081 / 1084          9.7         103.1       2.3X
    +        Native ORC Vectorized (0.5%) (Java)           1066 / 1069          9.8         101.6       2.3X
    +        Hive built-in ORC (0.5%)                      2928 / 2938          3.6         279.2       0.8X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.95%)                         1287 / 1331          8.2         122.7       1.0X
    +        Native ORC Vectorized (0.95%)                  404 /  407         26.0          38.5       3.2X
    +        Native ORC Vectorized (0.95%) (Java)           405 /  409         25.9          38.6       3.2X
    +        Hive built-in ORC (0.95%)                     1612 / 1644          6.5         153.7       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from wide table ($width columns)", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1116 / 1117          0.9        1064.7       1.0X
    +        Native ORC Vectorized                           92 /   98         11.3          88.2      12.1X
    +        Native ORC Vectorized (Java)                    90 /   96         11.6          86.0      12.4X
    +        Hive built-in ORC                              376 /  386          2.8         358.6       3.0X
    --- End diff --
    
    hmm, why is hive ORC much faster than the native ORC for this case?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85518 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85518/testReport)** for PR 19943 at commit [`9f225e2`](https://github.com/apache/spark/commit/9f225e23b562c5baa7d137b13c697cdeca5e9d43).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159624874
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    --- End diff --
    
    Right. It's fixed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85837 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85837/testReport)** for PR 19943 at commit [`2cf98b6`](https://github.com/apache/spark/commit/2cf98b6734c806f66e21df50520a465b03d9f060).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160072053
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    --- End diff --
    
    use for loop in java. please update all other places.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by henrify <gi...@git.apache.org>.
Github user henrify commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @dongjoon-hyun Ok thanks. It is pity that the single buffer cannot be used, would have reduced number of arraycopy() calls by 5 orders of magnitude.. Btw have you tested the inlining behaviour or tried to extract the copying loop of one type to a small separate method?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160164837
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    +      DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        putDecimalWritable(
    +          toColumn,
    +          index,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          data.vector[index]);
    +      }
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +  private void putValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putBoolean(index, vector[index] == 1);
    +        }
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putByte(index, (byte)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putShort(index, (short)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putInt(index, (int)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof LongType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +        }
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putFloat(index, (float)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof DoubleType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putDouble(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putByteArray(
    +            index, vector.vector[index], vector.start[index], vector.length[index]);
    +        }
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    --- End diff --
    
    ditto


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @henrify , @cloud-fan . For @henrify 's question, I got the answer. The answer is negative like the official document. Even ORC reader side,  the data for a VectorizedRowBatch comes from more than one internal buffer some cases. So, even ORC reader itself doesn't assume that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85845 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85845/testReport)** for PR 19943 at commit [`2cf98b6`](https://github.com/apache/spark/commit/2cf98b6734c806f66e21df50520a465b03d9f060).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159627447
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,357 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.Random
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     132 /  138        119.4           8.4       1.0X
    --- End diff --
    
    Yep. I'll add that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85518/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160089167
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,517 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1188 / 1230         13.2          75.6       1.0X
    +        Native ORC Vectorized                          163 /  174         96.7          10.3       7.3X
    +        Native ORC Vectorized (Java)                   156 /  168        100.8           9.9       7.6X
    +        Hive built-in ORC                             1413 / 1416         11.1          89.8       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1270 / 1324         12.4          80.7       1.0X
    +        Native ORC Vectorized                          160 /  166         98.2          10.2       7.9X
    +        Native ORC Vectorized (Java)                   160 /  169         98.2          10.2       7.9X
    +        Hive built-in ORC                             1662 / 1681          9.5         105.6       0.8X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1353 / 1365         11.6          86.0       1.0X
    +        Native ORC Vectorized                          260 /  274         60.4          16.5       5.2X
    +        Native ORC Vectorized (Java)                   225 /  235         69.8          14.3       6.0X
    +        Hive built-in ORC                             1908 / 1933          8.2         121.3       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1335 / 1357         11.8          84.9       1.0X
    +        Native ORC Vectorized                          288 /  302         54.7          18.3       4.6X
    +        Native ORC Vectorized (Java)                   292 /  296         53.9          18.5       4.6X
    +        Hive built-in ORC                             1908 / 1973          8.2         121.3       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1405 / 1469         11.2          89.3       1.0X
    +        Native ORC Vectorized                          361 /  363         43.6          22.9       3.9X
    +        Native ORC Vectorized (Java)                   324 /  332         48.6          20.6       4.3X
    +        Hive built-in ORC                             2044 / 2073          7.7         130.0       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1468 / 1482         10.7          93.3       1.0X
    +        Native ORC Vectorized                          395 /  403         39.8          25.1       3.7X
    +        Native ORC Vectorized (Java)                   397 /  406         39.6          25.2       3.7X
    +        Hive built-in ORC                             2078 / 2097          7.6         132.1       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2586 / 2670          4.1         246.6       1.0X
    +        Native ORC Vectorized                         1219 / 1228          8.6         116.3       2.1X
    +        Native ORC Vectorized (Java)                  1348 / 1358          7.8         128.6       1.9X
    --- End diff --
    
    @cloud-fan . There is some performance mystery here. So, I'm still having both one.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85743/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160168115
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1263 / 1296         12.5          80.3       1.0X
    +        Native ORC Vectorized                          159 /  166         98.6          10.1       7.9X
    +        Hive built-in ORC                             1513 / 1525         10.4          96.2       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1196 / 1232         13.1          76.1       1.0X
    +        Native ORC Vectorized                          163 /  168         96.7          10.3       7.4X
    +        Hive built-in ORC                             1625 / 1640          9.7         103.3       0.7X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1292 / 1378         12.2          82.2       1.0X
    +        Native ORC Vectorized                          228 /  236         68.9          14.5       5.7X
    +        Hive built-in ORC                             1829 / 1835          8.6         116.3       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1372 / 1398         11.5          87.2       1.0X
    +        Native ORC Vectorized                          286 /  300         55.1          18.2       4.8X
    +        Hive built-in ORC                             1911 / 1913          8.2         121.5       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1387 / 1415         11.3          88.2       1.0X
    +        Native ORC Vectorized                          326 /  329         48.2          20.7       4.3X
    +        Hive built-in ORC                             2004 / 2013          7.8         127.4       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1478 / 1524         10.6          94.0       1.0X
    +        Native ORC Vectorized                          412 /  416         38.2          26.2       3.6X
    +        Hive built-in ORC                             2070 / 2106          7.6         131.6       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2597 / 2671          4.0         247.7       1.0X
    +        Native ORC Vectorized                         1307 / 1315          8.0         124.7       2.0X
    +        Hive built-in ORC                             3867 / 3878          2.7         368.8       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR              1560 / 1562         10.1          99.2       1.0X
    +        Read data column - Native ORC Vectorized       294 /  301         53.4          18.7       5.3X
    +        Read data column - Hive built-in ORC          2101 / 2128          7.5         133.6       0.7X
    +        Read partition column - Native ORC MR         1080 / 1087         14.6          68.7       1.4X
    +        Read partition column - Native ORC Vectorized   54 /   58        289.5           3.5      28.7X
    +        Read partition column - Hive built-in ORC     1315 / 1316         12.0          83.6       1.2X
    +        Read both columns - Native ORC MR             1581 / 1591          9.9         100.5       1.0X
    +        Read both columns - Native ORC Vectorized      329 /  339         47.8          20.9       4.7X
    +        Read both columns - Hive built-in ORC         2124 / 2158          7.4         135.0       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1307 / 1309          8.0         124.6       1.0X
    +        Native ORC Vectorized                          327 /  336         32.1          31.2       4.0X
    +        Hive built-in ORC                             2009 / 2072          5.2         191.6       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.0%)                          2534 / 2535          4.1         241.7       1.0X
    +        Native ORC Vectorized (0.0%)                   961 /  963         10.9          91.6       2.6X
    +        Hive built-in ORC (0.0%)                      3980 / 4005          2.6         379.5       0.6X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.5%)                          2417 / 2422          4.3         230.5       1.0X
    +        Native ORC Vectorized (0.5%)                  1314 / 1315          8.0         125.3       1.8X
    +        Hive built-in ORC (0.5%)                      2940 / 2953          3.6         280.4       0.8X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.95%)                         1306 / 1318          8.0         124.5       1.0X
    +        Native ORC Vectorized (0.95%)                  521 /  539         20.1          49.7       2.5X
    +        Hive built-in ORC (0.95%)                     1636 / 1641          6.4         156.0       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from wide table ($width columns)", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    --- End diff --
    
    nit: `Single Column Scan from wide table`. The current title is too long, the result table is not well aligned.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078014
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    --- End diff --
    
    Done.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069601
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    --- End diff --
    
    This provides the same information as the variable name, we don't need this comment.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160169481
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1263 / 1296         12.5          80.3       1.0X
    +        Native ORC Vectorized                          159 /  166         98.6          10.1       7.9X
    +        Hive built-in ORC                             1513 / 1525         10.4          96.2       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1196 / 1232         13.1          76.1       1.0X
    +        Native ORC Vectorized                          163 /  168         96.7          10.3       7.4X
    +        Hive built-in ORC                             1625 / 1640          9.7         103.3       0.7X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1292 / 1378         12.2          82.2       1.0X
    +        Native ORC Vectorized                          228 /  236         68.9          14.5       5.7X
    +        Hive built-in ORC                             1829 / 1835          8.6         116.3       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1372 / 1398         11.5          87.2       1.0X
    +        Native ORC Vectorized                          286 /  300         55.1          18.2       4.8X
    +        Hive built-in ORC                             1911 / 1913          8.2         121.5       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1387 / 1415         11.3          88.2       1.0X
    +        Native ORC Vectorized                          326 /  329         48.2          20.7       4.3X
    +        Hive built-in ORC                             2004 / 2013          7.8         127.4       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1478 / 1524         10.6          94.0       1.0X
    +        Native ORC Vectorized                          412 /  416         38.2          26.2       3.6X
    +        Hive built-in ORC                             2070 / 2106          7.6         131.6       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2597 / 2671          4.0         247.7       1.0X
    +        Native ORC Vectorized                         1307 / 1315          8.0         124.7       2.0X
    +        Hive built-in ORC                             3867 / 3878          2.7         368.8       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR              1560 / 1562         10.1          99.2       1.0X
    +        Read data column - Native ORC Vectorized       294 /  301         53.4          18.7       5.3X
    +        Read data column - Hive built-in ORC          2101 / 2128          7.5         133.6       0.7X
    +        Read partition column - Native ORC MR         1080 / 1087         14.6          68.7       1.4X
    +        Read partition column - Native ORC Vectorized   54 /   58        289.5           3.5      28.7X
    +        Read partition column - Hive built-in ORC     1315 / 1316         12.0          83.6       1.2X
    +        Read both columns - Native ORC MR             1581 / 1591          9.9         100.5       1.0X
    +        Read both columns - Native ORC Vectorized      329 /  339         47.8          20.9       4.7X
    +        Read both columns - Hive built-in ORC         2124 / 2158          7.4         135.0       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1307 / 1309          8.0         124.6       1.0X
    +        Native ORC Vectorized                          327 /  336         32.1          31.2       4.0X
    +        Hive built-in ORC                             2009 / 2072          5.2         191.6       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.0%)                          2534 / 2535          4.1         241.7       1.0X
    +        Native ORC Vectorized (0.0%)                   961 /  963         10.9          91.6       2.6X
    +        Hive built-in ORC (0.0%)                      3980 / 4005          2.6         379.5       0.6X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.5%)                          2417 / 2422          4.3         230.5       1.0X
    +        Native ORC Vectorized (0.5%)                  1314 / 1315          8.0         125.3       1.8X
    +        Hive built-in ORC (0.5%)                      2940 / 2953          3.6         280.4       0.8X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR (0.95%)                         1306 / 1318          8.0         124.5       1.0X
    +        Native ORC Vectorized (0.95%)                  521 /  539         20.1          49.7       2.5X
    +        Hive built-in ORC (0.95%)                     1636 / 1641          6.4         156.0       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from wide table ($width columns)", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1119 / 1126          0.9        1067.4       1.0X
    +        Native ORC Vectorized                           93 /   99         11.3          88.3      12.1X
    +        Hive built-in ORC                              386 /  389          2.7         368.3       2.9X
    +
    +        SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2250 / 2271          0.5        2145.6       1.0X
    +        Native ORC Vectorized                          156 /  164          6.7         149.1      14.4X
    +        Hive built-in ORC                              589 /  601          1.8         561.7       3.8X
    +
    +        SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 3340 / 3360          0.3        3185.5       1.0X
    +        Native ORC Vectorized                          273 /  283          3.8         260.8      12.2X
    +        Hive built-in ORC                              851 /  861          1.2         811.4       3.9X
    --- End diff --
    
    @dongjoon-hyun are you saying `Hive built-in ORC` doesn't use the ORC MR reader and has a better implementation regarding column pruning?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160077793
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158764338
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    --- End diff --
    
    Can we merge two cases for `StringType` and `BinaryType` into one case?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85793/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160080614
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]);
    +                index += 1;
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendBoolean(data[index] == 1);
    --- End diff --
    
    good catch! Since we allocated a big enough batch in `initBatch`, I think here we should call the `putXXX` API.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156239181
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -378,6 +378,11 @@ object SQLConf {
         .checkValues(Set("hive", "native"))
         .createWithDefault("native")
     
    +  val ORC_VECTORIZED_READER_ENABLED = buildConf("spark.sql.orc.vectorizedReader.enabled")
    --- End diff --
    
    parquet config: `spark.sql.parquet.enableVectorizedReader`. Shall we have similar config name?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156239606
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -378,6 +378,11 @@ object SQLConf {
         .checkValues(Set("hive", "native"))
         .createWithDefault("native")
     
    +  val ORC_VECTORIZED_READER_ENABLED = buildConf("spark.sql.orc.vectorizedReader.enabled")
    --- End diff --
    
    Sure, @viirya .


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85745/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Could you be more specific, @henrify ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160316976
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,605 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +import java.util.stream.IntStream;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    +   * because they have different default values like the following.
    +   *
    +   * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024
    +   * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024
    +   */
    +  public static final int DEFAULT_SIZE = 4 * 1024;
    +
    +  /**
    +   * Returns the number of micros since epoch from an element of TimestampColumnVector.
    +   */
    +  private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) {
    +    return vector.time[index] * 1000L + vector.nanos[index] / 1000L;
    +  }
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return recordReader.getProgress();
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        putByteArrays(batchSize, toColumn, ((BytesColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      putNonNullBooleans(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof ByteType) {
    +      putNonNullBytes(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof ShortType) {
    +      putNonNullShorts(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      putNonNullInts(batchSize, (LongColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      putNonNullTimestamps(batchSize, (TimestampColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof FloatType) {
    +      putNonNullFloats(batchSize, (DoubleColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      putNonNullByteArray(batchSize, (BytesColumnVector) fromColumn, toColumn);
    +    } else if (type instanceof DecimalType) {
    +      putNonNullDecimals(batchSize, (DecimalColumnVector) fromColumn, toColumn, (DecimalType) type);
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +
    +  private void putValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      putBooleans(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof ByteType) {
    +      putBytes(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof ShortType) {
    +      putShorts(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      putInts(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof LongType) {
    +      putLongs(batchSize, (LongColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof TimestampType) {
    +      putTimestamps(batchSize, (TimestampColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof FloatType) {
    +      putFloats(batchSize, (DoubleColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof DoubleType) {
    +      putDoubles(batchSize, (DoubleColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      putByteArrays(batchSize, (BytesColumnVector)fromColumn, toColumn);
    +    } else if (type instanceof DecimalType) {
    +      putDecimals(batchSize, (DecimalColumnVector)fromColumn, toColumn, (DecimalType) type);
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +  // --------------------------------------------------------------------------
    +  // Put a value
    +  // --------------------------------------------------------------------------
    +
    +  private static void putDecimalWritable(
    +      WritableColumnVector toColumn,
    +      int index,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +      Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInt(index, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLong(index, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      arrayData.putBytes(index * 16, bytes.length, bytes, 0);
    +      toColumn.putArray(index, index * 16, bytes.length);
    +    }
    +  }
    +
    +  // --------------------------------------------------------------------------
    +  // Put repeating values
    +  // --------------------------------------------------------------------------
    +
    +  private void putByteArrays(int count, WritableColumnVector toColumn, byte[] bytes) {
    +    WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +    int size = bytes.length;
    +    arrayData.reserve(size);
    +    arrayData.putBytes(0, size, bytes, 0);
    +    for (int index = 0; index < count; index++) {
    +      toColumn.putArray(index, 0, size);
    +    }
    +  }
    +
    +  private static void putDecimalWritables(
    +      WritableColumnVector toColumn,
    +      int size,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +      Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInts(0, size, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLongs(0, size, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      arrayData.reserve(bytes.length);
    +      arrayData.putBytes(0, bytes.length, bytes, 0);
    +      for (int index = 0; index < size; index++) {
    +        toColumn.putArray(index, 0, bytes.length);
    +      }
    +    }
    +  }
    +
    +  // --------------------------------------------------------------------------
    +  // Put non-null values
    +  // --------------------------------------------------------------------------
    +
    +  private void putNonNullBooleans(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) {
    +    long[] data = fromColumn.vector;
    +    for (int index = 0; index < count; index++) {
    +      toColumn.putBoolean(index, data[index] == 1);
    +    }
    +  }
    +
    +  private void putNonNullByteArray(int count, BytesColumnVector fromColumn, WritableColumnVector toColumn) {
    +    BytesColumnVector data = fromColumn;
    +    WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +    int totalNumBytes = IntStream.of(data.length).sum();
    +    arrayData.reserve(totalNumBytes);
    +    for (int index = 0, pos = 0; index < count; pos += data.length[index], index++) {
    +      arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]);
    +      toColumn.putArray(index, pos, data.length[index]);
    +    }
    +  }
    +
    +  private void putNonNullBytes(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) {
    --- End diff --
    
    shall we revert these? Since they do not help with performance, I'd like to inline these small functions.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @cloud-fan .
    I rebased based on #20116, could you review this again?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85466 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85466/testReport)** for PR 19943 at commit [`9f225e2`](https://github.com/apache/spark/commit/9f225e23b562c5baa7d137b13c697cdeca5e9d43).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160016341
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,493 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.vectorized._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +    totalRowCount = reader.getNumberOfRows
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse, "No filters are allowed")
    --- End diff --
    
    does this mean we can't do filter push down?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159624804
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    --- End diff --
    
    I added a comment into `assert`. It means no filters are applied into this batch.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160236163
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    +      DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        putDecimalWritable(
    +          toColumn,
    +          index,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          data.vector[index]);
    +      }
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +  private void putValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putBoolean(index, vector[index] == 1);
    +        }
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putByte(index, (byte)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putShort(index, (short)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putInt(index, (int)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof LongType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +        }
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putFloat(index, (float)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof DoubleType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putDouble(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector vector = (BytesColumnVector)fromColumn;
    --- End diff --
    
    Yep. `putByteArray` is removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159220027
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    --- End diff --
    
    so it should be `Writable column vectors`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85827 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85827/testReport)** for PR 19943 at commit [`91b3d66`](https://github.com/apache/spark/commit/91b3d662fd99ad099b3d1226a8ecb261a6db0ae0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078135
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    --- End diff --
    
    Good idea. It's removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158768353
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala ---
    @@ -170,6 +171,8 @@ case class FileSourceScanExec(
     
       val needsUnsafeRowConversion: Boolean = if (relation.fileFormat.isInstanceOf[ParquetSource]) {
         SparkSession.getActiveSession.get.sessionState.conf.parquetVectorizedReaderEnabled
    +  } else if (relation.fileFormat.isInstanceOf[OrcFileFormat]) {
    +    SparkSession.getActiveSession.get.sessionState.conf.orcVectorizedReaderEnabled
    --- End diff --
    
    Different than Parquet, for now we enable vectorized ORC reader when batch output is supported. We don't need unsafe row conversion at all for ORC. Because once it supports batch, we go batch-based approach. If it doesn't support batch, we don't enable vectorized ORC reader at all, so we don't need unsafe row conversion too.
    
    Once we can enable vectorized ORC even batch is not supported, we need to add this.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85463/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160123375
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,510 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      toColumn.putByteArray(0, data.vector[0]);
    --- End diff --
    
    do you mean `data.vector[0]` contains all the bytes of this `BytesColumnVector`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158895321
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByte(data(index).toByte)
    +                index += 1
    +              }
    +            case ShortType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendShort(data(index).toShort)
    +                index += 1
    +              }
    +            case IntegerType | DateType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendInt(data(index).toInt)
    +                index += 1
    +              }
    +            case LongType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              toColumn.appendLongs(batchSize, data, 0)
    +
    +            case TimestampType =>
    +              val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendLong(fromTimestampColumnVector(data, index))
    +                index += 1
    +              }
    +
    +            case FloatType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendFloat(data(index).toFloat)
    +                index += 1
    +              }
    +            case DoubleType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              toColumn.appendDoubles(batchSize, data, 0)
    +
    +            case StringType =>
    +              val data = fromColumn.asInstanceOf[BytesColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(
    +                  data.vector(index), data.start(index), data.length(index))
    +                index += 1
    +              }
    +            case BinaryType =>
    +              val data = fromColumn.asInstanceOf[BytesColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector(index), data.start(index), data.length(index))
    +                index += 1
    +              }
    +
    +            case DecimalType.Fixed(precision, scale) =>
    +              val data = fromColumn.asInstanceOf[DecimalColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                val d = data.vector(index)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +                index += 1
    +              }
    +
    +            case dt =>
    +              throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +          }
    +        } else {
    +          for (index <- 0 until batchSize) {
    +            if (fromColumn.isNull(index)) {
    +              toColumn.appendNull()
    +            } else {
    +              field.dataType match {
    +                case BooleanType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) == 1
    +                  toColumn.appendBoolean(data)
    +
    +                case ByteType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toByte
    +                  toColumn.appendByte(data)
    +                case ShortType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toShort
    +                  toColumn.appendShort(data)
    +                case IntegerType | DateType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toInt
    +                  toColumn.appendInt(data)
    +                case LongType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index)
    +                  toColumn.appendLong(data)
    +
    +                case TimestampType =>
    +                  val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                  toColumn.appendLong(fromTimestampColumnVector(data, index))
    +
    +                case FloatType =>
    +                  val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index).toFloat
    +                  toColumn.appendFloat(data)
    +                case DoubleType =>
    +                  val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index)
    +                  toColumn.appendDouble(data)
    +
    +                case StringType =>
    +                  val v = fromColumn.asInstanceOf[BytesColumnVector]
    +                  toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index))
    +
    +                case BinaryType =>
    +                  val v = fromColumn.asInstanceOf[BytesColumnVector]
    +                  toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index))
    +
    +                case DecimalType.Fixed(precision, scale) =>
    +                  val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(index)
    +                  appendDecimalWritable(toColumn, precision, scale, d)
    +
    +                case dt =>
    +                  throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +              }
    +            }
    +          }
    +        }
    +      }
    +      i += 1
    +    }
    +    true
    +  }
    +}
    +
    +/**
    + * Constants for OrcColumnarBatchReader.
    + */
    +object OrcColumnarBatchReader {
    +  /**
    +   * Default memory mode for ColumnarBatch.
    +   */
    +  val DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP
    +
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    +   * because they have different default values like the following.
    +   *
    +   * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024
    +   * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024
    +   */
    +  val DEFAULT_SIZE: Int = 4 * 1024
    +
    +  /**
    +   * Returns the number of micros since epoch from an element of TimestampColumnVector.
    +   */
    +  private def fromTimestampColumnVector(vector: TimestampColumnVector, index: Int): Long =
    +    vector.time(index) * 1000L + vector.nanos(index) / 1000L
    +
    +  /**
    +   * Append a decimalWritable to a writableColumnVector.
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85827/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160087637
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredSchema.length() == requestedColIds.length);
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    --- End diff --
    
    do we need this? In `initBatch` we allocated a big enough batch, and `batchSize` here should always be smaller than the batch capacity.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85823 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85823/testReport)** for PR 19943 at commit [`8fc2162`](https://github.com/apache/spark/commit/8fc2162c3be968324c40a8717e4ddcc5cf173ec9).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160087436
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    --- End diff --
    
    Oh, sorry. I missed this. Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160207776
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    --- End diff --
    
    I'm a little confused, do you mean the ORC reader will skip empty batch(due to PPD) until it's the end?
    
    BTW does Hive report progress correctly?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160358917
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,523 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +import java.util.stream.IntStream;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    --- End diff --
    
    nit: We use this value for ORC reader to make it consistent with Spark's columnar batch, because their default batch sizes are different like the following.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160079405
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]);
    +                index += 1;
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendBoolean(data[index] == 1);
    +              index += 1;
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendByte((byte)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendShort((short)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendInt((int)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendLong(fromTimestampColumnVector(data, index));
    +              index += 1;
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendFloat((float)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendByteArray(data.vector[index], data.start[index], data.length[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                data.vector[index]);
    +              index += 1;
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        } else {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendBoolean(vector[index] == 1);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendByte((byte)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendShort((short)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendInt((int)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof LongType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendLong(vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendLong(fromTimestampColumnVector(vector, index));
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendFloat((float)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof DoubleType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendDouble(vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendByteArray(
    +                  vector.vector[index], vector.start[index], vector.length[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                appendDecimalWritable(
    +                  toColumn,
    +                  decimalType.precision(),
    +                  decimalType.scale(),
    +                  vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        }
    +      }
    +      i += 1;
    +    }
    +    return true;
    +  }
    +
    +  /**
    +   * Default memory mode for ColumnarBatch.
    +   */
    +  public static final MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP;
    --- End diff --
    
    I added like the above and didn't add `memoryMode` parameter at `initBatch`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    BTW @dongjoon-hyun can you also address https://github.com/apache/spark/pull/19943#discussion_r160076528 ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160017579
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,493 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.vectorized._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +    totalRowCount = reader.getNumberOfRows
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse, "No filters are allowed")
    --- End diff --
    
    I'll try to update the message, @HyukjinKwon .


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160089474
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    --- End diff --
    
    @cloud-fan . This happens when PPD.
    The total number of row shows all rows here. When PPD is applied, the next batch can be empty on the fly. ORC doesn't fill the unqualified rows into row batches.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85827 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85827/testReport)** for PR 19943 at commit [`91b3d66`](https://github.com/apache/spark/commit/91b3d662fd99ad099b3d1226a8ecb261a6db0ae0).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160306085
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName
    +  private val HIVE_ORC_FORMAT = classOf[org.apache.spark.sql.hive.orc.OrcFileFormat].getCanonicalName
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1180 / 1230         13.3          75.0       1.0X
    +        Native ORC Vectorized                          159 /  169         98.8          10.1       7.4X
    +        Hive built-in ORC                             1395 / 1396         11.3          88.7       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1216 / 1267         12.9          77.3       1.0X
    +        Native ORC Vectorized                          163 /  172         96.4          10.4       7.5X
    +        Hive built-in ORC                             1649 / 1672          9.5         104.8       0.7X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1331 / 1332         11.8          84.6       1.0X
    +        Native ORC Vectorized                          233 /  245         67.6          14.8       5.7X
    +        Hive built-in ORC                             1832 / 1839          8.6         116.5       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1382 / 1389         11.4          87.8       1.0X
    +        Native ORC Vectorized                          291 /  299         54.1          18.5       4.8X
    +        Hive built-in ORC                             1926 / 1936          8.2         122.5       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1399 / 1478         11.2          88.9       1.0X
    +        Native ORC Vectorized                          324 /  329         48.6          20.6       4.3X
    +        Hive built-in ORC                             1938 / 1945          8.1         123.2       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1441 / 1470         10.9          91.6       1.0X
    +        Native ORC Vectorized                          406 /  408         38.8          25.8       3.6X
    +        Hive built-in ORC                             2031 / 2039          7.7         129.1       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2716 / 2738          3.9         259.0       1.0X
    +        Native ORC Vectorized                         1325 / 1325          7.9         126.4       2.0X
    +        Hive built-in ORC                             3607 / 3645          2.9         344.0       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR               1544 / 1547         10.2          98.1       1.0X
    +        Read data column - Native ORC Vectorized        298 /  306         52.7          19.0       5.2X
    +        Read data column - Hive built-in ORC           2089 / 2097          7.5         132.8       0.7X
    +        Read partition column - Native ORC MR          1050 / 1051         15.0          66.8       1.5X
    +        Read partition column - Native ORC Vectorized    54 /   57        290.0           3.4      28.5X
    +        Read partition column - Hive built-in ORC      1271 / 1280         12.4          80.8       1.2X
    +        Read both columns - Native ORC MR              1572 / 1605         10.0         100.0       1.0X
    +        Read both columns - Native ORC Vectorized       332 /  338         47.4          21.1       4.6X
    +        Read both columns - Hive built-in ORC          2108 / 2123          7.5         134.0       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def repeatedStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Repeated String", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Repeated String:                         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1297 / 1327          8.1         123.7       1.0X
    +        Native ORC Vectorized                          317 /  327         33.1          30.2       4.1X
    +        Hive built-in ORC                             1970 / 1973          5.3         187.9       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark(s"String with Nulls Scan ($fractionOfNulls%)", values)
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan (0.0%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2531 / 2542          4.1         241.4       1.0X
    +        Native ORC Vectorized                          947 /  952         11.1          90.3       2.7X
    +        Hive built-in ORC                             4012 / 4034          2.6         382.6       0.6X
    +
    +        String with Nulls Scan (0.5%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2388 / 2407          4.4         227.8       1.0X
    +        Native ORC Vectorized                         1235 / 1236          8.5         117.8       1.9X
    +        Hive built-in ORC                             2951 / 2958          3.6         281.4       0.8X
    +
    +        String with Nulls Scan (0.95%):          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1325 / 1346          7.9         126.4       1.0X
    +        Native ORC Vectorized                          460 /  468         22.8          43.9       2.9X
    +        Hive built-in ORC                             1600 / 1607          6.6         152.6       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from $width columns", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from 100 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1107 / 1118          0.9        1056.1       1.0X
    +        Native ORC Vectorized                           94 /  100         11.1          89.8      11.8X
    +        Hive built-in ORC                              382 /  390          2.7         364.0       2.9X
    +
    +        SQL Single Column Scan from 200 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2278 / 2287          0.5        2172.0       1.0X
    +        Native ORC Vectorized                          158 /  165          6.6         150.6      14.4X
    +        Hive built-in ORC                              585 /  590          1.8         557.7       3.9X
    +
    +        SQL Single Column Scan from 300 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 3386 / 3394          0.3        3229.1       1.0X
    +        Native ORC Vectorized                          271 /  281          3.9         258.2      12.5X
    +        Hive built-in ORC                              843 /  852          1.2         803.6       4.0X
    --- End diff --
    
    don't forget this question :)  https://github.com/apache/spark/pull/19943#discussion_r160169481


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159628611
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    recordReader.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType | DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType | BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByte(data(index).toByte)
    +                index += 1
    +              }
    +            case ShortType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendShort(data(index).toShort)
    +                index += 1
    +              }
    +            case IntegerType | DateType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendInt(data(index).toInt)
    +                index += 1
    +              }
    +            case LongType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              toColumn.appendLongs(batchSize, data, 0)
    +
    +            case TimestampType =>
    +              val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendLong(fromTimestampColumnVector(data, index))
    +                index += 1
    +              }
    +
    +            case FloatType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendFloat(data(index).toFloat)
    +                index += 1
    +              }
    +            case DoubleType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              toColumn.appendDoubles(batchSize, data, 0)
    +
    +            case StringType | BinaryType =>
    +              val data = fromColumn.asInstanceOf[BytesColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector(index), data.start(index), data.length(index))
    +                index += 1
    +              }
    +
    +            case DecimalType.Fixed(precision, scale) =>
    +              val data = fromColumn.asInstanceOf[DecimalColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                appendDecimalWritable(toColumn, precision, scale, data.vector(index))
    +                index += 1
    +              }
    +
    +            case dt =>
    +              throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +          }
    +        } else {
    +          for (index <- 0 until batchSize) {
    +            if (fromColumn.isNull(index)) {
    +              toColumn.appendNull()
    +            } else {
    +              field.dataType match {
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160017338
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala ---
    @@ -139,15 +146,25 @@ class OrcFileFormat
           }
         }
     
    +    val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
    +    val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled &&
    +      supportBatch(sparkSession, resultSchema)
    --- End diff --
    
    Right. It's fixed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Thank you for review, @cloud-fan, @viirya, @kiszk, @HyukjinKwon, @henrify.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160117940
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,510 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    +      DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        putDecimalWritable(
    +          toColumn,
    +          index,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          data.vector[index]);
    +      }
    +    } else {
    +      throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +    }
    +  }
    +
    +  private void putValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putBoolean(index, vector[index] == 1);
    +        }
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putByte(index, (byte)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putShort(index, (short)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putInt(index, (int)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof LongType) {
    +      long[] vector = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +        }
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putFloat(index, (float)vector[index]);
    +        }
    +      }
    +    } else if (type instanceof DoubleType) {
    +      double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putDouble(index, vector[index]);
    +        }
    +      }
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +      for (int index = 0; index < batchSize; index++) {
    +        if (fromColumn.isNull[index]) {
    +          toColumn.putNull(index);
    +        } else {
    +          toColumn.putByteArray(
    +            index, vector.vector[index], vector.start[index], vector.length[index]);
    --- End diff --
    
    Thanks, @henrify .


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85788/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159624817
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85763/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156474306
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala ---
    @@ -139,15 +146,25 @@ class OrcFileFormat
           }
         }
     
    +    val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
    +    val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled &&
    +      supportBatch(sparkSession, resultSchema)
    --- End diff --
    
    Ur, it seems to make the implementation complicated in order to provide slower version.
    If you don't mind, I'll proceed that in a next follow-up PR.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #84733 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84733/testReport)** for PR 19943 at commit [`449a0a2`](https://github.com/apache/spark/commit/449a0a2f8f58b28bc93837645ee542b555f4e6c3).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #84809 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84809/testReport)** for PR 19943 at commit [`d302259`](https://github.com/apache/spark/commit/d30225993c2c674898ea43e64b555b1283ca8e0c).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160124118
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,510 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      toColumn.putByteArray(0, data.vector[0]);
    --- End diff --
    
    I found this. Previously, I missed this from `BytesColumnVector` doc.
    ```
     * This class supports string and binary data by value reference -- i.e. each field is
     * explicitly present, as opposed to provided by a dictionary reference.
     * In some cases, all the values will be in the same byte array to begin with,
     * but this need not be the case. If each value is in a separate byte
     * array to start with, or not all of the values are in the same original
     * byte array, you can still assign data by reference into this column vector.
     * This gives flexibility to use this in multiple situations.
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160071875
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    --- End diff --
    
    how about we follow parquet and call it `initBatch`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069594
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    --- End diff --
    
    `... of the result columnar batch`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by henrify <gi...@git.apache.org>.
Github user henrify commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078679
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]);
    +                index += 1;
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendBoolean(data[index] == 1);
    --- End diff --
    
    Actually, you know the number of rows in advance. Wouldn't it possible to call reserve() once, and then use the putX() API instead of appendX() API inside the loops? That should be significantly faster.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by mmccline <gi...@git.apache.org>.
Github user mmccline commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156475822
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L)
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale)
    +                value.changePrecision(precision, scale)
    +                if (precision <= Decimal.MAX_INT_DIGITS) {
    +                  toColumn.appendInts(batchSize, value.toUnscaledLong.toInt)
    +                } else if (precision <= Decimal.MAX_LONG_DIGITS) {
    +                  toColumn.appendLongs(batchSize, value.toUnscaledLong)
    +                } else {
    +                  val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray
    +                  var index = 0
    +                  while (index < batchSize) {
    +                    toColumn.appendByteArray(bytes, 0, bytes.length)
    +                    index += 1
    +                  }
    +                }
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    --- End diff --
    
    Throwing data error exceptions can be nasty -- perhaps setting the value to a known value and a warning might be better.  I noticed leveraging of not nullable in the Hive optimizer recently but we will see if there are any practical issues that arise...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by mmccline <gi...@git.apache.org>.
Github user mmccline commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156232929
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L)
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale)
    +                value.changePrecision(precision, scale)
    +                if (precision <= Decimal.MAX_INT_DIGITS) {
    +                  toColumn.appendInts(batchSize, value.toUnscaledLong.toInt)
    +                } else if (precision <= Decimal.MAX_LONG_DIGITS) {
    +                  toColumn.appendLongs(batchSize, value.toUnscaledLong)
    +                } else {
    +                  val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray
    +                  var index = 0
    +                  while (index < batchSize) {
    +                    toColumn.appendByteArray(bytes, 0, bytes.length)
    +                    index += 1
    +                  }
    +                }
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    --- End diff --
    
    I'm not sure I get how this if stmt works right.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85788 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85788/testReport)** for PR 19943 at commit [`3a0702a`](https://github.com/apache/spark/commit/3a0702ae0b31f762c9f3da06d267a02ec8d1a23b).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160090596
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredSchema.length() == requestedColIds.length);
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.putNulls(0, batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              for (int index = 0; index < batchSize; index++) {
    +                toColumn.putByteArray(index, data.vector[0], data.start[0], data.length[0]);
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              putDecimalWritables(
    +                toColumn,
    +                batchSize,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putBoolean(index, data[index] == 1);
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByte(index, (byte)data[index]);
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putShort(index, (short)data[index]);
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putInt(index, (int)data[index]);
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putFloat(index, (float)data[index]);
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              putDecimalWritable(
    +                toColumn,
    +                index,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                data.vector[index]);
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        } else {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putBoolean(index, vector[index] == 1);
    +              }
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putByte(index, (byte)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putShort(index, (short)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putInt(index, (int)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof LongType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putLong(index, vector[index]);
    +              }
    +            }
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putLong(index, fromTimestampColumnVector(vector, index));
    +              }
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putFloat(index, (float)vector[index]);
    +              }
    +            }
    +          } else if (type instanceof DoubleType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putDouble(index, vector[index]);
    +              }
    +            }
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                toColumn.putByteArray(
    +                  index, vector.vector[index], vector.start[index], vector.length[index]);
    +              }
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.putNull(index);
    +              } else {
    +                putDecimalWritable(
    +                  toColumn,
    +                  index,
    +                  decimalType.precision(),
    +                  decimalType.scale(),
    +                  vector[index]);
    +              }
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    +   * because they have different default values like the following.
    +   *
    +   * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024
    +   * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024
    +   */
    +  public static final int DEFAULT_SIZE = 4 * 1024;
    +
    +  /**
    +   * Returns the number of micros since epoch from an element of TimestampColumnVector.
    +   */
    +  private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) {
    +    return vector.time[index] * 1000L + vector.nanos[index] / 1000L;
    +  }
    +
    +  /**
    +   * Put a `HiveDecimalWritable` to a `WritableColumnVector`.
    +   */
    +  private static void putDecimalWritable(
    +      WritableColumnVector toColumn,
    +      int index,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +      Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    +    value.changePrecision(precision, scale);
    +
    +    if (precision <= Decimal.MAX_INT_DIGITS()) {
    +      toColumn.putInt(index, (int) value.toUnscaledLong());
    +    } else if (precision <= Decimal.MAX_LONG_DIGITS()) {
    +      toColumn.putLong(index, value.toUnscaledLong());
    +    } else {
    +      byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray();
    +      toColumn.putByteArray(index, bytes, 0, bytes.length);
    +    }
    +  }
    +
    +  /**
    +   * Put `HiveDecimalWritable`s to a `WritableColumnVector`.
    +   */
    +  private static void putDecimalWritables(
    +      WritableColumnVector toColumn,
    +      int size,
    +      int precision,
    +      int scale,
    +      HiveDecimalWritable decimalWritable) {
    +    HiveDecimal decimal = decimalWritable.getHiveDecimal();
    +    Decimal value =
    +            Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale());
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19943
  
     overall looks good, my major concern is https://github.com/apache/spark/pull/19943/files#r159221758 , do you have an answer? This may be a big drawback compared to the wrapper solution.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069559
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    --- End diff --
    
    This provides the same information as the variable name, we don't need this comment.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160071984
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    --- End diff --
    
    this can be `requiredFields: Array[StructField]`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160232913
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    --- End diff --
    
    Thank you for the tip. I'll update like the following. `putBytes` needs to keep the current byte position `pos`.
    ```scala
    -      for (int index = 0; index < batchSize; index++) {
    -        toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      int totalNumBytes = IntStream.of(data.length).sum();
    +      arrayData.reserve(totalNumBytes);
    +      for (int index = 0, pos = 0; index < batchSize; index++, pos += data.length[index]) {
    +        arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]);
    +        toColumn.putArray(index, pos, data.length[index]);
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160017493
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          156 /  163        100.7           9.9       1.0X
    +        Native ORC MR                                 1222 / 1236         12.9          77.7       0.1X
    +        Hive built-in ORC                             1572 / 1625         10.0         100.0       0.1X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          215 /  225         73.1          13.7       1.0X
    +        Native ORC MR                                 1337 / 1458         11.8          85.0       0.2X
    +        Hive built-in ORC                             1696 / 1707          9.3         107.8       0.1X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          285 /  294         55.2          18.1       1.0X
    +        Native ORC MR                                 1397 / 1419         11.3          88.8       0.2X
    +        Hive built-in ORC                             2086 / 2143          7.5         132.6       0.1X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          341 /  350         46.1          21.7       1.0X
    +        Native ORC MR                                 1461 / 1492         10.8          92.9       0.2X
    +        Hive built-in ORC                             2002 / 2095          7.9         127.3       0.2X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          347 /  356         45.3          22.1       1.0X
    +        Native ORC MR                                 1524 / 1553         10.3          96.9       0.2X
    +        Hive built-in ORC                             2393 / 2404          6.6         152.1       0.1X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          403 /  411         39.0          25.6       1.0X
    +        Native ORC MR                                 1517 / 1529         10.4          96.5       0.3X
    +        Hive built-in ORC                             2054 / 2134          7.7         130.6       0.2X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                         1382 / 1400          7.6         131.8       1.0X
    +        Native ORC MR                                 2689 / 2765          3.9         256.4       0.5X
    +        Hive built-in ORC                             3889 / 3894          2.7         370.9       0.4X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC Vectorized       347 /  350         45.3          22.1       1.0X
    +        Read data column - Native ORC MR              1611 / 1613          9.8         102.4       0.2X
    +        Read data column - Hive built-in ORC          2082 / 2091          7.6         132.4       0.2X
    +        Read partition column - Native ORC Vectorized   55 /   57        286.5           3.5       6.3X
    +        Read partition column - Native ORC MR         1062 / 1063         14.8          67.5       0.3X
    +        Read partition column - Hive built-in ORC     1334 / 1334         11.8          84.8       0.3X
    +        Read both columns - Native ORC Vectorized      380 /  388         41.3          24.2       0.9X
    +        Read both columns - Native ORC MR             1654 / 1672          9.5         105.2       0.2X
    +        Read both columns - Hive built-in ORC         2209 / 2209          7.1         140.5       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized                          406 /  414         25.8          38.7       1.0X
    +        Native ORC MR                                 1372 / 1381          7.6         130.8       0.3X
    +        Hive built-in ORC                             2016 / 2036          5.2         192.2       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC Vectorized (0.0%)                  1122 / 1129          9.3         107.0       1.0X
    +        Native ORC MR (0.0%)                          2551 / 2619          4.1         243.3       0.4X
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160077716
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -386,6 +386,16 @@ object SQLConf {
         .checkValues(Set("hive", "native"))
         .createWithDefault("native")
     
    +  val ORC_VECTORIZED_READER_ENABLED = buildConf("spark.sql.orc.enableVectorizedReader")
    +    .doc("Enables vectorized orc decoding.")
    +    .booleanConf
    +    .createWithDefault(true)
    +
    +  val ORC_VECTORIZED_JAVA_READER_ENABLED = buildConf("spark.sql.orc.enableVectorizedJavaReader")
    --- End diff --
    
    Yes. It does, but we need to choose the best one for both performance and maintenance perspective.
    
    Until now, according to the performance result, java implementations doesn't outperform scala version. I'll remove at the last minute to make it sure.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84809/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160017293
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,493 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.vectorized._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +    totalRowCount = reader.getNumberOfRows
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse, "No filters are allowed")
    --- End diff --
    
    Can we maybe make this message a bit better or leave a small comment on this?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160082708
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]);
    +                index += 1;
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendBoolean(data[index] == 1);
    --- End diff --
    
    Thank you, @henrify . I'll update like that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159220570
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    --- End diff --
    
    what does this mean? we can add an error message to this `assert` as a comment.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069664
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    --- End diff --
    
    nit: always write
    ```
    for (...) {
      ...
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85837/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160305663
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,528 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +import java.util.stream.IntStream;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +        int size = data.vector[0].length;
    +        arrayData.reserve(size);
    +        arrayData.putBytes(0, size, data.vector[0], 0);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, 0, size);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +      int totalNumBytes = IntStream.of(data.length).sum();
    +      arrayData.reserve(totalNumBytes);
    +      for (int index = 0, pos = 0; index < batchSize; pos += data.length[index], index++) {
    +        arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]);
    +        toColumn.putArray(index, pos, data.length[index]);
    +      }
    +    } else if (type instanceof DecimalType) {
    +      DecimalType decimalType = (DecimalType)type;
    +      DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +      if (decimalType.precision() > Decimal.MAX_INT_DIGITS()) {
    --- End diff --
    
    type: MAX_LONG_DIGITS


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Thank you, @cloud-fan . I'll try to update this after #20116 lands on master branch.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/19943


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078288
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    --- End diff --
    
    Correct.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85456 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85456/testReport)** for PR 19943 at commit [`3e1d479`](https://github.com/apache/spark/commit/3e1d479196dfcb21e2d5f641a50c0b663b8247a1).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85463 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85463/testReport)** for PR 19943 at commit [`3e1d479`](https://github.com/apache/spark/commit/3e1d479196dfcb21e2d5f641a50c0b663b8247a1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158768650
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    --- End diff --
    
    `columnVectors` -> `ColumnVector`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158764605
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByte(data(index).toByte)
    +                index += 1
    +              }
    +            case ShortType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendShort(data(index).toShort)
    +                index += 1
    +              }
    +            case IntegerType | DateType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendInt(data(index).toInt)
    +                index += 1
    +              }
    +            case LongType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              toColumn.appendLongs(batchSize, data, 0)
    +
    +            case TimestampType =>
    +              val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendLong(fromTimestampColumnVector(data, index))
    +                index += 1
    +              }
    +
    +            case FloatType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    ditto


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158895416
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByte(data(index).toByte)
    +                index += 1
    +              }
    +            case ShortType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendShort(data(index).toShort)
    +                index += 1
    +              }
    +            case IntegerType | DateType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendInt(data(index).toInt)
    +                index += 1
    +              }
    +            case LongType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              toColumn.appendLongs(batchSize, data, 0)
    +
    +            case TimestampType =>
    +              val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendLong(fromTimestampColumnVector(data, index))
    +                index += 1
    +              }
    +
    +            case FloatType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendFloat(data(index).toFloat)
    +                index += 1
    +              }
    +            case DoubleType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              toColumn.appendDoubles(batchSize, data, 0)
    +
    +            case StringType =>
    +              val data = fromColumn.asInstanceOf[BytesColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(
    +                  data.vector(index), data.start(index), data.length(index))
    +                index += 1
    +              }
    +            case BinaryType =>
    +              val data = fromColumn.asInstanceOf[BytesColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector(index), data.start(index), data.length(index))
    +                index += 1
    +              }
    +
    +            case DecimalType.Fixed(precision, scale) =>
    +              val data = fromColumn.asInstanceOf[DecimalColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                val d = data.vector(index)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +                index += 1
    +              }
    +
    +            case dt =>
    +              throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +          }
    +        } else {
    +          for (index <- 0 until batchSize) {
    +            if (fromColumn.isNull(index)) {
    +              toColumn.appendNull()
    +            } else {
    +              field.dataType match {
    +                case BooleanType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) == 1
    +                  toColumn.appendBoolean(data)
    +
    +                case ByteType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toByte
    +                  toColumn.appendByte(data)
    +                case ShortType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toShort
    +                  toColumn.appendShort(data)
    +                case IntegerType | DateType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toInt
    +                  toColumn.appendInt(data)
    +                case LongType =>
    +                  val data = fromColumn.asInstanceOf[LongColumnVector].vector(index)
    +                  toColumn.appendLong(data)
    +
    +                case TimestampType =>
    +                  val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                  toColumn.appendLong(fromTimestampColumnVector(data, index))
    +
    +                case FloatType =>
    +                  val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index).toFloat
    +                  toColumn.appendFloat(data)
    +                case DoubleType =>
    +                  val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index)
    +                  toColumn.appendDouble(data)
    +
    +                case StringType =>
    +                  val v = fromColumn.asInstanceOf[BytesColumnVector]
    +                  toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index))
    +
    +                case BinaryType =>
    --- End diff --
    
    Yep. I merged them.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160163934
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    --- End diff --
    
    `putByteArray` calls `appendXXX` under the hood. For better performance, we should avoid using it.
    ```
    val arrayData = toColumn.getChildColumn(0)
    val totalNumBytes = data.length.sum // We could use Java 8 stream API here
    arrayData.reserve(totalNumBytes)
    for (int index = 0; index < batchSize; index++) {
      arrayData.putBytes(...)
      toColumn.putArray(...)
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078262
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160090565
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,482 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructType requiredSchema;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructType requiredSchema,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredSchema.length() == requestedColIds.length);
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +        toColumn.reserve(batchSize);
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.putNulls(0, batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              for (int index = 0; index < batchSize; index++) {
    +                toColumn.putByteArray(index, data.vector[0], data.start[0], data.length[0]);
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              putDecimalWritables(
    +                toColumn,
    +                batchSize,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putBoolean(index, data[index] == 1);
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByte(index, (byte)data[index]);
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putShort(index, (short)data[index]);
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putInt(index, (int)data[index]);
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putFloat(index, (float)data[index]);
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            for (int index = 0; index < batchSize; index++) {
    +              toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]);
    +            }
    +          } else if (type instanceof DecimalType) {
    --- End diff --
    
    Did I miss something in [Line 290](https://github.com/apache/spark/pull/19943/files/3a0702ae0b31f762c9f3da06d267a02ec8d1a23b#diff-d0cab20acc0e0c3b27ec27b479e0f912R290)?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078799
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    --- End diff --
    
    Correct. It's removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160167182
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1263 / 1296         12.5          80.3       1.0X
    +        Native ORC Vectorized                          159 /  166         98.6          10.1       7.9X
    +        Hive built-in ORC                             1513 / 1525         10.4          96.2       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1196 / 1232         13.1          76.1       1.0X
    +        Native ORC Vectorized                          163 /  168         96.7          10.3       7.4X
    +        Hive built-in ORC                             1625 / 1640          9.7         103.3       0.7X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1292 / 1378         12.2          82.2       1.0X
    +        Native ORC Vectorized                          228 /  236         68.9          14.5       5.7X
    +        Hive built-in ORC                             1829 / 1835          8.6         116.3       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1372 / 1398         11.5          87.2       1.0X
    +        Native ORC Vectorized                          286 /  300         55.1          18.2       4.8X
    +        Hive built-in ORC                             1911 / 1913          8.2         121.5       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1387 / 1415         11.3          88.2       1.0X
    +        Native ORC Vectorized                          326 /  329         48.2          20.7       4.3X
    +        Hive built-in ORC                             2004 / 2013          7.8         127.4       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1478 / 1524         10.6          94.0       1.0X
    +        Native ORC Vectorized                          412 /  416         38.2          26.2       3.6X
    +        Hive built-in ORC                             2070 / 2106          7.6         131.6       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2597 / 2671          4.0         247.7       1.0X
    +        Native ORC Vectorized                         1307 / 1315          8.0         124.7       2.0X
    +        Hive built-in ORC                             3867 / 3878          2.7         368.8       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR              1560 / 1562         10.1          99.2       1.0X
    +        Read data column - Native ORC Vectorized       294 /  301         53.4          18.7       5.3X
    +        Read data column - Hive built-in ORC          2101 / 2128          7.5         133.6       0.7X
    +        Read partition column - Native ORC MR         1080 / 1087         14.6          68.7       1.4X
    +        Read partition column - Native ORC Vectorized   54 /   58        289.5           3.5      28.7X
    +        Read partition column - Hive built-in ORC     1315 / 1316         12.0          83.6       1.2X
    +        Read both columns - Native ORC MR             1581 / 1591          9.9         100.5       1.0X
    +        Read both columns - Native ORC Vectorized      329 /  339         47.8          20.9       4.7X
    +        Read both columns - Hive built-in ORC         2124 / 2158          7.4         135.0       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    --- End diff --
    
    why call it `String Dictionary`? How can we make sure the data is dictionary encoded in the ORC file?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160087485
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    --- End diff --
    
    initially we set up `totoalRowCount` in `initiliaze`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160316107
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,605 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +import java.util.stream.IntStream;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +  /**
    +   * The default size of batch. We use this value for both ORC and Spark consistently
    +   * because they have different default values like the following.
    +   *
    +   * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024
    +   * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024
    +   */
    +  public static final int DEFAULT_SIZE = 4 * 1024;
    +
    +  /**
    +   * Returns the number of micros since epoch from an element of TimestampColumnVector.
    +   */
    +  private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) {
    +    return vector.time[index] * 1000L + vector.nanos[index] / 1000L;
    +  }
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    --- End diff --
    
    Do we still need these 2? They are not used for progress reporting anymore, and we don't rely on them to detect the end of file.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160317124
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName
    +  private val HIVE_ORC_FORMAT = classOf[org.apache.spark.sql.hive.orc.OrcFileFormat].getCanonicalName
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1202 / 1265         13.1          76.4       1.0X
    +        Native ORC Vectorized                          162 /  172         97.1          10.3       7.4X
    +        Hive built-in ORC                             1410 / 1428         11.2          89.6       0.9X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1266 / 1286         12.4          80.5       1.0X
    +        Native ORC Vectorized                          165 /  174         95.2          10.5       7.7X
    +        Hive built-in ORC                             1701 / 1704          9.2         108.2       0.7X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1307 / 1307         12.0          83.1       1.0X
    +        Native ORC Vectorized                          232 /  248         67.9          14.7       5.6X
    +        Hive built-in ORC                             1793 / 1793          8.8         114.0       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1360 / 1372         11.6          86.5       1.0X
    +        Native ORC Vectorized                          293 /  303         53.8          18.6       4.7X
    +        Hive built-in ORC                             1913 / 1933          8.2         121.6       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1389 / 1488         11.3          88.3       1.0X
    +        Native ORC Vectorized                          340 /  346         46.2          21.6       4.1X
    +        Hive built-in ORC                             1976 / 1997          8.0         125.6       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1461 / 1465         10.8          92.9       1.0X
    +        Native ORC Vectorized                          395 /  406         39.8          25.1       3.7X
    +        Hive built-in ORC                             2127 / 2146          7.4         135.2       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2843 / 2935          3.7         271.1       1.0X
    +        Native ORC Vectorized                         1349 / 1359          7.8         128.6       2.1X
    +        Hive built-in ORC                             3862 / 3881          2.7         368.3       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR               1545 / 1568         10.2          98.2       1.0X
    +        Read data column - Native ORC Vectorized        300 /  304         52.4          19.1       5.1X
    +        Read data column - Hive built-in ORC           2097 / 2117          7.5         133.3       0.7X
    +        Read partition column - Native ORC MR          1023 / 1026         15.4          65.1       1.5X
    +        Read partition column - Native ORC Vectorized    54 /   56        292.3           3.4      28.7X
    +        Read partition column - Hive built-in ORC      1285 / 1289         12.2          81.7       1.2X
    +        Read both columns - Native ORC MR              1564 / 1565         10.1          99.4       1.0X
    +        Read both columns - Native ORC Vectorized       336 /  340         46.8          21.4       4.6X
    +        Read both columns - Hive built-in ORC          2100 / 2123          7.5         133.5       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def repeatedStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Repeated String", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Repeated String:                         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1338 / 1340          7.8         127.6       1.0X
    +        Native ORC Vectorized                          342 /  350         30.6          32.6       3.9X
    +        Hive built-in ORC                             2036 / 2117          5.2         194.2       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark(s"String with Nulls Scan ($fractionOfNulls%)", values)
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan (0.0%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2510 / 2527          4.2         239.3       1.0X
    +        Native ORC Vectorized                         1012 / 1012         10.4          96.5       2.5X
    +        Hive built-in ORC                             3967 / 3984          2.6         378.3       0.6X
    +
    +        String with Nulls Scan (0.5%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2374 / 2388          4.4         226.4       1.0X
    +        Native ORC Vectorized                         1269 / 1275          8.3         121.0       1.9X
    +        Hive built-in ORC                             2994 / 2998          3.5         285.5       0.8X
    +
    +        String with Nulls Scan (0.95%):          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1285 / 1312          8.2         122.6       1.0X
    +        Native ORC Vectorized                          500 /  508         21.0          47.7       2.6X
    +        Hive built-in ORC                             1630 / 1638          6.4         155.5       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from $width columns", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from 100 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1107 / 1111          0.9        1055.4       1.0X
    +        Native ORC Vectorized                           93 /  102         11.3          88.8      11.9X
    +        Hive built-in ORC                              377 /  389          2.8         359.6       2.9X
    +
    +        SQL Single Column Scan from 200 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2258 / 2270          0.5        2153.8       1.0X
    +        Native ORC Vectorized                          160 /  168          6.6         152.6      14.1X
    +        Hive built-in ORC                              591 /  597          1.8         563.7       3.8X
    +
    +        SQL Single Column Scan from 300 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 3364 / 3391          0.3        3208.3       1.0X
    +        Native ORC Vectorized                          273 /  284          3.8         260.1      12.3X
    +        Hive built-in ORC                              831 /  842          1.3         792.8       4.0X
    --- End diff --
    
    don't forget this question :)  https://github.com/apache/spark/pull/19943#discussion_r160169481
      


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160123996
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,517 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1161 / 1168         13.5          73.8       1.0X
    +        Native ORC Vectorized                          163 /  171         96.3          10.4       7.1X
    +        Native ORC Vectorized (Java)                   155 /  163        101.6           9.8       7.5X
    +        Hive built-in ORC                             1427 / 1427         11.0          90.7       0.8X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1261 / 1321         12.5          80.2       1.0X
    +        Native ORC Vectorized                          160 /  167         98.2          10.2       7.9X
    +        Native ORC Vectorized (Java)                   160 /  167         98.4          10.2       7.9X
    +        Hive built-in ORC                             1655 / 1687          9.5         105.2       0.8X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1369 / 1449         11.5          87.1       1.0X
    +        Native ORC Vectorized                          263 /  277         59.8          16.7       5.2X
    +        Native ORC Vectorized (Java)                   225 /  237         70.0          14.3       6.1X
    +        Hive built-in ORC                             1867 / 1899          8.4         118.7       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1472 / 1474         10.7          93.6       1.0X
    +        Native ORC Vectorized                          289 /  300         54.5          18.4       5.1X
    +        Native ORC Vectorized (Java)                   286 /  294         54.9          18.2       5.1X
    +        Hive built-in ORC                             1917 / 1934          8.2         121.9       0.8X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1484 / 1484         10.6          94.3       1.0X
    +        Native ORC Vectorized                          365 /  370         43.1          23.2       4.1X
    +        Native ORC Vectorized (Java)                   326 /  335         48.2          20.7       4.5X
    +        Hive built-in ORC                             1978 / 2049          8.0         125.8       0.8X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1550 / 1554         10.1          98.6       1.0X
    +        Native ORC Vectorized                          396 /  405         39.7          25.2       3.9X
    +        Native ORC Vectorized (Java)                   394 /  402         39.9          25.1       3.9X
    +        Hive built-in ORC                             2072 / 2084          7.6         131.8       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2636 / 2734          4.0         251.4       1.0X
    +        Native ORC Vectorized                         1267 / 1267          8.3         120.9       2.1X
    +        Native ORC Vectorized (Java)                  1182 / 1183          8.9         112.7       2.2X
    +        Hive built-in ORC                             3724 / 3764          2.8         355.2       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized (Java)") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR                    1587 / 1592          9.9         100.9       1.0X
    +        Read data column - Native ORC Vectorized             290 /  309         54.3          18.4       5.5X
    +        Read data column - Native ORC Vectorized (Java)      293 /  297         53.7          18.6       5.4X
    +        Read data column - Hive built-in ORC                2204 / 2214          7.1         140.1       0.7X
    +        Read partition column - Native ORC MR               1078 / 1097         14.6          68.5       1.5X
    +        Read partition column - Native ORC Vectorized         53 /   56        294.0           3.4      29.7X
    +        Read partition column - Native ORC Vectorized (Java)  52 /   55        300.7           3.3      30.4X
    +        Read partition column - Hive built-in ORC           1279 / 1287         12.3          81.3       1.2X
    +        Read both columns - Native ORC MR                   1665 / 1674          9.4         105.9       1.0X
    +        Read both columns - Native ORC Vectorized            327 /  333         48.0          20.8       4.8X
    +        Read both columns - Native ORC Vectorized (Java)     327 /  332         48.2          20.8       4.9X
    +        Read both columns - Hive built-in ORC               2157 / 2169          7.3         137.1       0.7X
    --- End diff --
    
    This should be 3 benchmarks.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85829/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85793 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85793/testReport)** for PR 19943 at commit [`15cac9c`](https://github.com/apache/spark/commit/15cac9cf6b99415b03fc818fbb14a16b722c9058).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160325249
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName
    +  private val HIVE_ORC_FORMAT = classOf[org.apache.spark.sql.hive.orc.OrcFileFormat].getCanonicalName
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1202 / 1265         13.1          76.4       1.0X
    +        Native ORC Vectorized                          162 /  172         97.1          10.3       7.4X
    +        Hive built-in ORC                             1410 / 1428         11.2          89.6       0.9X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1266 / 1286         12.4          80.5       1.0X
    +        Native ORC Vectorized                          165 /  174         95.2          10.5       7.7X
    +        Hive built-in ORC                             1701 / 1704          9.2         108.2       0.7X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1307 / 1307         12.0          83.1       1.0X
    +        Native ORC Vectorized                          232 /  248         67.9          14.7       5.6X
    +        Hive built-in ORC                             1793 / 1793          8.8         114.0       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1360 / 1372         11.6          86.5       1.0X
    +        Native ORC Vectorized                          293 /  303         53.8          18.6       4.7X
    +        Hive built-in ORC                             1913 / 1933          8.2         121.6       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1389 / 1488         11.3          88.3       1.0X
    +        Native ORC Vectorized                          340 /  346         46.2          21.6       4.1X
    +        Hive built-in ORC                             1976 / 1997          8.0         125.6       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1461 / 1465         10.8          92.9       1.0X
    +        Native ORC Vectorized                          395 /  406         39.8          25.1       3.7X
    +        Hive built-in ORC                             2127 / 2146          7.4         135.2       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2843 / 2935          3.7         271.1       1.0X
    +        Native ORC Vectorized                         1349 / 1359          7.8         128.6       2.1X
    +        Hive built-in ORC                             3862 / 3881          2.7         368.3       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR               1545 / 1568         10.2          98.2       1.0X
    +        Read data column - Native ORC Vectorized        300 /  304         52.4          19.1       5.1X
    +        Read data column - Hive built-in ORC           2097 / 2117          7.5         133.3       0.7X
    +        Read partition column - Native ORC MR          1023 / 1026         15.4          65.1       1.5X
    +        Read partition column - Native ORC Vectorized    54 /   56        292.3           3.4      28.7X
    +        Read partition column - Hive built-in ORC      1285 / 1289         12.2          81.7       1.2X
    +        Read both columns - Native ORC MR              1564 / 1565         10.1          99.4       1.0X
    +        Read both columns - Native ORC Vectorized       336 /  340         46.8          21.4       4.6X
    +        Read both columns - Hive built-in ORC          2100 / 2123          7.5         133.5       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def repeatedStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Repeated String", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Repeated String:                         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1338 / 1340          7.8         127.6       1.0X
    +        Native ORC Vectorized                          342 /  350         30.6          32.6       3.9X
    +        Hive built-in ORC                             2036 / 2117          5.2         194.2       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark(s"String with Nulls Scan ($fractionOfNulls%)", values)
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan (0.0%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2510 / 2527          4.2         239.3       1.0X
    +        Native ORC Vectorized                         1012 / 1012         10.4          96.5       2.5X
    +        Hive built-in ORC                             3967 / 3984          2.6         378.3       0.6X
    +
    +        String with Nulls Scan (0.5%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2374 / 2388          4.4         226.4       1.0X
    +        Native ORC Vectorized                         1269 / 1275          8.3         121.0       1.9X
    +        Hive built-in ORC                             2994 / 2998          3.5         285.5       0.8X
    +
    +        String with Nulls Scan (0.95%):          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1285 / 1312          8.2         122.6       1.0X
    +        Native ORC Vectorized                          500 /  508         21.0          47.7       2.6X
    +        Hive built-in ORC                             1630 / 1638          6.4         155.5       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from $width columns", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from 100 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1107 / 1111          0.9        1055.4       1.0X
    +        Native ORC Vectorized                           93 /  102         11.3          88.8      11.9X
    +        Hive built-in ORC                              377 /  389          2.8         359.6       2.9X
    +
    +        SQL Single Column Scan from 200 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2258 / 2270          0.5        2153.8       1.0X
    +        Native ORC Vectorized                          160 /  168          6.6         152.6      14.1X
    +        Hive built-in ORC                              591 /  597          1.8         563.7       3.8X
    +
    +        SQL Single Column Scan from 300 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 3364 / 3391          0.3        3208.3       1.0X
    +        Native ORC Vectorized                          273 /  284          3.8         260.1      12.3X
    +        Hive built-in ORC                              831 /  842          1.3         792.8       4.0X
    --- End diff --
    
    Sorry for late response.
    
    As we know, ORC read path evolves in two ways; ORC MR reader and ORC VectorizedRowBatch. But, the core part is ORC VectorizedRowBatch. ORC MR reader is reimplemented as a wrapper of the ORC VectorizedRowBatch.
    
    What I mentioned previously was the difference between ORC VecterizedRowBatch and ORC MR reader (not about Hive 1.2.1 ORC read path). Maybe, the performance difference between Hive 1.2.1 ORC read path and ORC MR reader is due to the redevelopment of ORC MR reader.
    
    Anyway, I consider both ORC MR and ORC Vectorized version as a single `Native ORC`. So, I described in a vague way sometimes. If this PR is merged, Spark is going to have both native ORC implementations (MR/Vectorized). Like Parquet, we will call vectorized version as Spark's native ORC.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158823225
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByte(data(index).toByte)
    +                index += 1
    +              }
    +            case ShortType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendShort(data(index).toShort)
    +                index += 1
    +              }
    +            case IntegerType | DateType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendInt(data(index).toInt)
    +                index += 1
    +              }
    +            case LongType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              toColumn.appendLongs(batchSize, data, 0)
    +
    +            case TimestampType =>
    +              val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendLong(fromTimestampColumnVector(data, index))
    +                index += 1
    +              }
    +
    +            case FloatType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendFloat(data(index).toFloat)
    +                index += 1
    +              }
    +            case DoubleType =>
    +              val data = fromColumn.asInstanceOf[DoubleColumnVector].vector
    +              toColumn.appendDoubles(batchSize, data, 0)
    +
    +            case StringType =>
    +              val data = fromColumn.asInstanceOf[BytesColumnVector]
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(
    +                  data.vector(index), data.start(index), data.length(index))
    +                index += 1
    +              }
    +            case BinaryType =>
    --- End diff --
    
    Can we merge `StringType` and `BinaryType` into one `case`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158816046
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    --- End diff --
    
    Simply `(DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP)` ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Oops. Without noticing your comments, I pushed another refactoring which split the functions.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078213
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    --- End diff --
    
    Thanks!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158764569
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    ditto


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85772 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85772/testReport)** for PR 19943 at commit [`7214ec0`](https://github.com/apache/spark/commit/7214ec03f8e48d51e0fd1f3314a0af6ac8275412).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by henrify <gi...@git.apache.org>.
Github user henrify commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    @dongjoon-hyun Thank you for testing the split methods. If anything the benchmark results look couple of percent slower now? Oh well, at least it is good to know that your code is as fast as it can be!
    
    I have no further ideas how performance could possibly be improved. Just many thanks to you and all reviewers for your hard work on this PR!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160124013
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,510 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public JavaOrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    +        for (int index = 0; index < batchSize; index++) {
    +          toColumn.putArray(index, data.start[0], data.length[0]);
    +        }
    +      } else if (type instanceof DecimalType) {
    +        DecimalType decimalType = (DecimalType)type;
    +        putDecimalWritables(
    +          toColumn,
    +          batchSize,
    +          decimalType.precision(),
    +          decimalType.scale(),
    +          ((DecimalColumnVector)fromColumn).vector[0]);
    +      } else {
    +        throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +      }
    +    }
    +  }
    +
    +  private void putNonNullValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    DataType type = field.dataType();
    +    if (type instanceof BooleanType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putBoolean(index, data[index] == 1);
    +      }
    +    } else if (type instanceof ByteType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putByte(index, (byte)data[index]);
    +      }
    +    } else if (type instanceof ShortType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putShort(index, (short)data[index]);
    +      }
    +    } else if (type instanceof IntegerType || type instanceof DateType) {
    +      long[] data = ((LongColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putInt(index, (int)data[index]);
    +      }
    +    } else if (type instanceof LongType) {
    +      toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof TimestampType) {
    +      TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putLong(index, fromTimestampColumnVector(data, index));
    +      }
    +    } else if (type instanceof FloatType) {
    +      double[] data = ((DoubleColumnVector)fromColumn).vector;
    +      for (int index = 0; index < batchSize; index++) {
    +        toColumn.putFloat(index, (float)data[index]);
    +      }
    +    } else if (type instanceof DoubleType) {
    +      toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +    } else if (type instanceof StringType || type instanceof BinaryType) {
    +      BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +      toColumn.putByteArray(0, data.vector[0]);
    --- End diff --
    
    It's for testing. Sorry, I'll revert this.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159222649
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala ---
    @@ -139,15 +146,25 @@ class OrcFileFormat
           }
         }
     
    +    val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
    +    val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled &&
    +      supportBatch(sparkSession, resultSchema)
    --- End diff --
    
    `supportBatch` already have the condition `conf.orcVectorizedReaderEnabled`, I think here we can just write
    ```
    val enableVectorizedReader = supportBatch(sparkSession, resultSchema)
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158895242
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    --- End diff --
    
    Yep!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160016446
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala ---
    @@ -110,4 +107,23 @@ object OrcUtils extends Logging {
           }
         }
       }
    +
    +  /**
    +   * Return a fixed ORC schema with data schema information, if needed.
    +   * The schema inside old ORC files might consist of invalid column names like '_col0'.
    +   */
    +  def getFixedTypeDescription(
    --- End diff --
    
    Do we really need this? The ORC schema is used to create the ORC batch, for the batch data I think we only care about the data types not field names.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159627960
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var recordReader: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (recordReader != null) {
    +      recordReader.close()
    +      recordReader = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    recordReader = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    recordReader.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    --- End diff --
    
    For a constant value, ORC fills it as repeating. It depends on data.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85744 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85744/testReport)** for PR 19943 at commit [`aeb6abd`](https://github.com/apache/spark/commit/aeb6abd66ee3338635edf9dca85894c14a05fb72).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85463 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85463/testReport)** for PR 19943 at commit [`3e1d479`](https://github.com/apache/spark/commit/3e1d479196dfcb21e2d5f641a50c0b663b8247a1).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156553867
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L)
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale)
    +                value.changePrecision(precision, scale)
    +                if (precision <= Decimal.MAX_INT_DIGITS) {
    +                  toColumn.appendInts(batchSize, value.toUnscaledLong.toInt)
    +                } else if (precision <= Decimal.MAX_LONG_DIGITS) {
    +                  toColumn.appendLongs(batchSize, value.toUnscaledLong)
    +                } else {
    +                  val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray
    +                  var index = 0
    +                  while (index < batchSize) {
    +                    toColumn.appendByteArray(bytes, 0, bytes.length)
    +                    index += 1
    +                  }
    +                }
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    --- End diff --
    
    Oh, I missed your comment here, @mmccline . Thank you for review!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85466 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85466/testReport)** for PR 19943 at commit [`9f225e2`](https://github.com/apache/spark/commit/9f225e23b562c5baa7d137b13c697cdeca5e9d43).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156241777
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala ---
    @@ -110,4 +107,21 @@ object OrcUtils extends Logging {
           }
         }
       }
    +
    +  /**
    +   * Return a fixed ORC schema with data schema information, if needed.
    --- End diff --
    
    Maybe explain what the issue to fix is?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84785/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160174950
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      StructField field = requiredFields[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] >= 0) {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          putRepeatingValues(batchSize, field, fromColumn, toColumn);
    +        } else if (fromColumn.noNulls) {
    +          putNonNullValues(batchSize, field, fromColumn, toColumn);
    +        } else {
    +          putValues(batchSize, field, fromColumn, toColumn);
    +        }
    +      }
    +    }
    +    return true;
    +  }
    +
    +  private void putRepeatingValues(
    +      int batchSize,
    +      StructField field,
    +      ColumnVector fromColumn,
    +      WritableColumnVector toColumn) {
    +    if (fromColumn.isNull[0]) {
    +      toColumn.putNulls(0, batchSize);
    +    } else {
    +      DataType type = field.dataType();
    +      if (type instanceof BooleanType) {
    +        toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +      } else if (type instanceof ByteType) {
    +        toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof ShortType) {
    +        toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof IntegerType || type instanceof DateType) {
    +        toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof LongType) {
    +        toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof TimestampType) {
    +        toColumn.putLongs(0, batchSize,
    +          fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +      } else if (type instanceof FloatType) {
    +        toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof DoubleType) {
    +        toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +      } else if (type instanceof StringType || type instanceof BinaryType) {
    +        BytesColumnVector data = (BytesColumnVector)fromColumn;
    +        toColumn.putByteArray(0, data.vector[0]);
    --- End diff --
    
    So, do you mean?
    ```
    -        toColumn.putByteArray(0, data.vector[0]);
    +        WritableColumnVector arrayData = toColumn.getChildColumn(0);
    +        arrayData.reserve(data.vector[0].length);
    +        arrayData.putBytes(0, data.vector[0].length, data.vector[0], 0);
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160078425
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    --- End diff --
    
    Oops. I'll rewrite the others, too.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by mmccline <gi...@git.apache.org>.
Github user mmccline commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156233426
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    --- End diff --
    
    The VectorizedRowBatch processing here is ignoring boolean batch.selectedInUse and int[] batch.selected array logical batch index processing.  If you are going to assume batch.selectedInUse is false for batches read from ORC, it ought to be asserted.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Also cc @kiszk , this question also applies to the table cache reader. We should think more about using a wrapper or writing to spark column vector.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069644
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    --- End diff --
    
    4 space identation.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85808/testReport)** for PR 19943 at commit [`ba03d20`](https://github.com/apache/spark/commit/ba03d20ac6c826b5f16307884e34c1f4022eb814).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #84809 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84809/testReport)** for PR 19943 at commit [`d302259`](https://github.com/apache/spark/commit/d30225993c2c674898ea43e64b555b1283ca8e0c).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Thanks. I checked the as-is inline behavior. As you told, ORC nextBatch is not inlined so far while Parquet nextBatch does. I'll try to optimize that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159221957
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,357 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.Random
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     132 /  138        119.4           8.4       1.0X
    --- End diff --
    
    can we compare `native ORC`, `vectorized native ORC` and `Hive built-in ORC`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85794/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069906
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    --- End diff --
    
    how would this happen?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160305118
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java ---
    @@ -0,0 +1,509 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `initBatch` should be called sequentially.
    + */
    +public class OrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  // ORC File Reader
    +  private Reader reader;
    +
    +  // Vectorized ORC Row Batch
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * The column IDs of the physical ORC file schema which are required by this reader.
    +   * -1 means this required column doesn't exist in the ORC file.
    +   */
    +  private int[] requestedColIds;
    +
    +  // Record reader from ORC row batch.
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  private StructField[] requiredFields;
    +
    +  // The result columnar batch for vectorized execution by whole-stage codegen.
    +  private ColumnarBatch columnarBatch;
    +
    +  // Writable column vectors of the result columnar batch.
    +  private WritableColumnVector[] columnVectors;
    +
    +  // The number of rows read and considered to be returned.
    +  private long rowsReturned = 0L;
    +
    +  private long totalRowCount = 0L;
    +
    +  /**
    +   * The memory mode of the columnarBatch
    +   */
    +  private final MemoryMode MEMORY_MODE;
    +
    +  public OrcColumnarBatchReader(boolean useOffHeap) {
    +    MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
    +  }
    +
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `initBatch` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +      OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Initialize columnar batch by setting required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void initBatch(
    +      TypeDescription orcSchema,
    +      int[] requestedColIds,
    +      StructField[] requiredFields,
    +      StructType partitionSchema,
    +      InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    this.requiredFields = requiredFields;
    +    this.requestedColIds = requestedColIds;
    +    assert(requiredFields.length == requestedColIds.length);
    +
    +    StructType resultSchema = new StructType(requiredFields);
    +    for (StructField f : partitionSchema.fields()) {
    +      resultSchema = resultSchema.add(f);
    +    }
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredFields.length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredFields.length; i++) {
    +      if (requestedColIds[i] == -1) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    --- End diff --
    
    I see. Parquet is fine as parquet reader in Spark only do row-group PPD, not record level PPD. I'm OK to leave it since progress reporting is not very important, but it would be great to take a look at how hive solves this issue.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156471417
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala ---
    @@ -139,15 +146,25 @@ class OrcFileFormat
           }
         }
     
    +    val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
    +    val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled &&
    +      supportBatch(sparkSession, resultSchema)
    --- End diff --
    
    Yep. I'll try.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85745 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85745/testReport)** for PR 19943 at commit [`aeb6abd`](https://github.com/apache/spark/commit/aeb6abd66ee3338635edf9dca85894c14a05fb72).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159224131
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,357 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.Random
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
    +  private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     132 /  138        119.4           8.4       1.0X
    +        Hive built-in ORC                             1328 / 1333         11.8          84.5       0.1X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     178 /  188         88.2          11.3       1.0X
    +        Hive built-in ORC                             1541 / 1560         10.2          98.0       0.1X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     242 /  256         64.9          15.4       1.0X
    +        Hive built-in ORC                             1650 / 1676          9.5         104.9       0.1X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     297 /  309         53.0          18.9       1.0X
    +        Hive built-in ORC                             1750 / 1766          9.0         111.3       0.2X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     352 /  363         44.7          22.4       1.0X
    +        Hive built-in ORC                             1749 / 1764          9.0         111.2       0.2X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     436 /  456         36.1          27.7       1.0X
    +        Hive built-in ORC                             1852 / 1860          8.5         117.8       0.2X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                    1374 / 1376          7.6         131.0       1.0X
    +        Hive built-in ORC                             3653 / 3664          2.9         348.4       0.4X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC                  321 /  327         49.0          20.4       1.0X
    +        Read data column - Hive built-in ORC          2041 / 2176          7.7         129.8       0.2X
    +        Read partition column - Native ORC              53 /   57        298.2           3.4       6.1X
    +        Read partition column - Hive built-in ORC     1176 / 1183         13.4          74.7       0.3X
    +        Read both columns - Native ORC                 335 /  340         47.0          21.3       1.0X
    +        Read both columns - Hive built-in ORC         1970 / 1974          8.0         125.2       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringDictionaryScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("String Dictionary", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String Dictionary:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC                                     363 /  382         28.9          34.7       1.0X
    +        Hive built-in ORC                             2012 / 2080          5.2         191.9       0.2X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark("String with Nulls Scan", values)
    +
    +        benchmark.addCase(s"Native ORC ($fractionOfNulls%)") { iter =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { iter =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC (0.0%)                             1120 / 1142          9.4         106.8       1.0X
    +        Hive built-in ORC (0.0%)                      4232 / 4284          2.5         403.6       0.3X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC (0.5%)                             1474 / 1512          7.1         140.5       1.0X
    +        Hive built-in ORC (0.5%)                      3114 / 3140          3.4         297.0       0.5X
    +
    +        String with Nulls Scan:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC (0.95%)                             568 /  589         18.5          54.1       1.0X
    +        Hive built-in ORC (0.95%)                     1548 / 1549          6.8         147.6       0.4X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan FROM $width-Column Rows", values)
    --- End diff --
    
    `Single Column Scan FROM wide table`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r156472027
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala ---
    @@ -139,15 +146,25 @@ class OrcFileFormat
           }
         }
     
    +    val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
    +    val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled &&
    +      supportBatch(sparkSession, resultSchema)
    +
         val broadcastedConf =
           sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
         val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
     
         (file: PartitionedFile) => {
           val conf = broadcastedConf.value.value
     
    +      val filePath = new Path(new URI(file.filePath))
    +
    +      val fs = filePath.getFileSystem(conf)
    +      val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
    +      val reader = OrcFile.createReader(filePath, readerOptions)
    --- End diff --
    
    The reader is used here, too. This extraction prevents redundant creation of the reader.
    ```scala
    batchReader.setRequiredSchema(
      OrcUtils.getFixedTypeDescription(reader.getSchema, dataSchema),
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160072105
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private ColumnarBatch columnarBatch;
    +
    +  /**
    +   * Writable column vectors of ColumnarBatch.
    +   */
    +  private WritableColumnVector[] columnVectors;
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private long rowsReturned = 0L;
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private long totalRowCount = 0L;
    +
    +  @Override
    +  public Void getCurrentKey() throws IOException, InterruptedException {
    +    return null;
    +  }
    +
    +  @Override
    +  public ColumnarBatch getCurrentValue() throws IOException, InterruptedException {
    +    return columnarBatch;
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    return nextBatch();
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    if (columnarBatch != null) {
    +      columnarBatch.close();
    +      columnarBatch = null;
    +    }
    +    if (recordReader != null) {
    +      recordReader.close();
    +      recordReader = null;
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    FileSplit fileSplit = (FileSplit)inputSplit;
    +    Configuration conf = taskAttemptContext.getConfiguration();
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath(),
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath().getFileSystem(conf)));
    +
    +    Reader.Options options =
    +            OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength());
    +    recordReader = reader.rows(options);
    +    totalRowCount = reader.getNumberOfRows();
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  public void setRequiredSchema(
    +    TypeDescription orcSchema,
    +    int[] requestedColIds,
    +    StructType requiredSchema,
    +    StructType partitionSchema,
    +    InternalRow partitionValues) {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE);
    +    assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`.
    +
    +    StructType resultSchema = new StructType(requiredSchema.fields());
    +    for (StructField f : partitionSchema.fields())
    +      resultSchema = resultSchema.add(f);
    +    this.requiredSchema = requiredSchema;
    +    this.requestedColIds = requestedColIds;
    +
    +    int capacity = DEFAULT_SIZE;
    +    if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema);
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity);
    +
    +    if (partitionValues.numFields() > 0) {
    +      int partitionIdx = requiredSchema.fields().length;
    +      for (int i = 0; i < partitionValues.numFields(); i++) {
    +        ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i);
    +        columnVectors[i + partitionIdx].setIsConstant();
    +      }
    +    }
    +
    +    // Initialize the missing columns once.
    +    for (int i = 0; i < requiredSchema.length(); i++) {
    +      if (requestedColIds[i] < 0) {
    +        columnVectors[i].putNulls(0, columnarBatch.capacity());
    +        columnVectors[i].setIsConstant();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private boolean nextBatch() throws IOException {
    +    if (rowsReturned >= totalRowCount) {
    +      return false;
    +    }
    +
    +    recordReader.nextBatch(batch);
    +    int batchSize = batch.size;
    +    if (batchSize == 0) {
    +      return false;
    +    }
    +    rowsReturned += batchSize;
    +    for (WritableColumnVector vector : columnVectors) {
    +      vector.reset();
    +    }
    +    columnarBatch.setNumRows(batchSize);
    +    int i = 0;
    +    while (i < requiredSchema.length()) {
    +      StructField field = requiredSchema.fields()[i];
    +      WritableColumnVector toColumn = columnVectors[i];
    +
    +      if (requestedColIds[i] < 0) {
    +        toColumn.appendNulls(batchSize);
    +      } else {
    +        ColumnVector fromColumn = batch.cols[requestedColIds[i]];
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull[0]) {
    +            toColumn.appendNulls(batchSize);
    +          } else {
    +            DataType type = field.dataType();
    +            if (type instanceof BooleanType) {
    +              toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1);
    +            } else if (type instanceof ByteType) {
    +              toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof ShortType) {
    +              toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof IntegerType || type instanceof DateType) {
    +              toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof LongType) {
    +              toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof TimestampType) {
    +              toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0));
    +            } else if (type instanceof FloatType) {
    +              toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof DoubleType) {
    +              toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]);
    +            } else if (type instanceof StringType || type instanceof BinaryType) {
    +              BytesColumnVector data = (BytesColumnVector)fromColumn;
    +              int index = 0;
    +              while (index < batchSize) {
    +                toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]);
    +                index += 1;
    +              }
    +            } else if (type instanceof DecimalType) {
    +              DecimalType decimalType = (DecimalType)type;
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                ((DecimalColumnVector)fromColumn).vector[0]);
    +            } else {
    +              throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +            }
    +          }
    +        } else if (fromColumn.noNulls) {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendBoolean(data[index] == 1);
    +              index += 1;
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendByte((byte)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendShort((short)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] data = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendInt((int)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof LongType) {
    +            toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector data = ((TimestampColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendLong(fromTimestampColumnVector(data, index));
    +              index += 1;
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] data = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendFloat((float)data[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof DoubleType) {
    +            toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector, 0);
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector data = ((BytesColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              toColumn.appendByteArray(data.vector[index], data.start[index], data.length[index]);
    +              index += 1;
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            DecimalColumnVector data = ((DecimalColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              appendDecimalWritable(
    +                toColumn,
    +                decimalType.precision(),
    +                decimalType.scale(),
    +                data.vector[index]);
    +              index += 1;
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        } else {
    +          DataType type = field.dataType();
    +          if (type instanceof BooleanType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendBoolean(vector[index] == 1);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof ByteType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendByte((byte)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof ShortType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendShort((short)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof IntegerType || type instanceof DateType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendInt((int)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof LongType) {
    +            long[] vector = ((LongColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendLong(vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof TimestampType) {
    +            TimestampColumnVector vector = ((TimestampColumnVector)fromColumn);
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendLong(fromTimestampColumnVector(vector, index));
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof FloatType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendFloat((float)vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof DoubleType) {
    +            double[] vector = ((DoubleColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendDouble(vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof StringType || type instanceof BinaryType) {
    +            BytesColumnVector vector = (BytesColumnVector)fromColumn;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                toColumn.appendByteArray(
    +                  vector.vector[index], vector.start[index], vector.length[index]);
    +              }
    +              index += 1;
    +            }
    +          } else if (type instanceof DecimalType) {
    +            DecimalType decimalType = (DecimalType)type;
    +            HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector;
    +            int index = 0;
    +            while (index < batchSize) {
    +              if (fromColumn.isNull[index]) {
    +                toColumn.appendNull();
    +              } else {
    +                appendDecimalWritable(
    +                  toColumn,
    +                  decimalType.precision(),
    +                  decimalType.scale(),
    +                  vector[index]);
    +              }
    +              index += 1;
    +            }
    +          } else {
    +            throw new UnsupportedOperationException("Unsupported Data Type: " + type);
    +          }
    +        }
    +      }
    +      i += 1;
    +    }
    +    return true;
    +  }
    +
    +  /**
    +   * Default memory mode for ColumnarBatch.
    +   */
    +  public static final MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP;
    --- End diff --
    
    don't do this, pass a `memoryMode` parameter in `initBatch`, like parquet


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158764558
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    Can we use `WritableColumn.appendBooleans()`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158893248
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    --- End diff --
    
    Thank you for decision, @cloud-fan . If then, I'll try to update the PR.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85808 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85808/testReport)** for PR 19943 at commit [`ba03d20`](https://github.com/apache/spark/commit/ba03d20ac6c826b5f16307884e34c1f4022eb814).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069577
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    --- End diff --
    
    `The result columnar batch for ...`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160077924
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    --- End diff --
    
    Updated.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84788/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160326383
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.File
    +
    +import scala.util.{Random, Try}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.{DataFrame, SparkSession}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{Benchmark, Utils}
    +
    +
    +/**
    + * Benchmark to measure ORC read performance.
    + *
    + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources.
    + */
    +// scalastyle:off line.size.limit
    +object OrcReadBenchmark {
    +  val conf = new SparkConf()
    +  conf.set("orc.compression", "snappy")
    +
    +  private val spark = SparkSession.builder()
    +    .master("local[1]")
    +    .appName("OrcReadBenchmark")
    +    .config(conf)
    +    .getOrCreate()
    +
    +  // Set default configs. Individual cases will change them if necessary.
    +  spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
    +
    +  def withTempPath(f: File => Unit): Unit = {
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    try f(path) finally Utils.deleteRecursively(path)
    +  }
    +
    +  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
    +    try f finally tableNames.foreach(spark.catalog.dropTempView)
    +  }
    +
    +  def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    +    val (keys, values) = pairs.unzip
    +    val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption)
    +    (keys, values).zipped.foreach(spark.conf.set)
    +    try f finally {
    +      keys.zip(currentValues).foreach {
    +        case (key, Some(value)) => spark.conf.set(key, value)
    +        case (key, None) => spark.conf.unset(key)
    +      }
    +    }
    +  }
    +
    +  private val NATIVE_ORC_FORMAT = classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName
    +  private val HIVE_ORC_FORMAT = classOf[org.apache.spark.sql.hive.orc.OrcFileFormat].getCanonicalName
    +
    +  private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = {
    +    val dirORC = dir.getCanonicalPath
    +
    +    if (partition.isDefined) {
    +      df.write.partitionBy(partition.get).orc(dirORC)
    +    } else {
    +      df.write.orc(dirORC)
    +    }
    +
    +    spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable")
    +    spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable")
    +  }
    +
    +  def numericScanBenchmark(values: Int, dataType: DataType): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single TINYINT Column Scan:          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1202 / 1265         13.1          76.4       1.0X
    +        Native ORC Vectorized                          162 /  172         97.1          10.3       7.4X
    +        Hive built-in ORC                             1410 / 1428         11.2          89.6       0.9X
    +
    +        SQL Single SMALLINT Column Scan:         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1266 / 1286         12.4          80.5       1.0X
    +        Native ORC Vectorized                          165 /  174         95.2          10.5       7.7X
    +        Hive built-in ORC                             1701 / 1704          9.2         108.2       0.7X
    +
    +        SQL Single INT Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1307 / 1307         12.0          83.1       1.0X
    +        Native ORC Vectorized                          232 /  248         67.9          14.7       5.6X
    +        Hive built-in ORC                             1793 / 1793          8.8         114.0       0.7X
    +
    +        SQL Single BIGINT Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1360 / 1372         11.6          86.5       1.0X
    +        Native ORC Vectorized                          293 /  303         53.8          18.6       4.7X
    +        Hive built-in ORC                             1913 / 1933          8.2         121.6       0.7X
    +
    +        SQL Single FLOAT Column Scan:            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1389 / 1488         11.3          88.3       1.0X
    +        Native ORC Vectorized                          340 /  346         46.2          21.6       4.1X
    +        Hive built-in ORC                             1976 / 1997          8.0         125.6       0.7X
    +
    +        SQL Single DOUBLE Column Scan:           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1461 / 1465         10.8          92.9       1.0X
    +        Native ORC Vectorized                          395 /  406         39.8          25.1       3.7X
    +        Hive built-in ORC                             2127 / 2146          7.4         135.2       0.7X
    +        */
    +        sqlBenchmark.run()
    +      }
    +    }
    +  }
    +
    +  def intStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Int and String Scan", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Int and String Scan:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2843 / 2935          3.7         271.1       1.0X
    +        Native ORC Vectorized                         1349 / 1359          7.8         128.6       2.1X
    +        Hive built-in ORC                             3862 / 3881          2.7         368.3       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def partitionTableScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Partitioned Table", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p"))
    +
    +        benchmark.addCase("Read data column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read data column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read data column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read partition column - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read partition column - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p) FROM hiveOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Read both columns - Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Read both columns - Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Read data column - Native ORC MR               1545 / 1568         10.2          98.2       1.0X
    +        Read data column - Native ORC Vectorized        300 /  304         52.4          19.1       5.1X
    +        Read data column - Hive built-in ORC           2097 / 2117          7.5         133.3       0.7X
    +        Read partition column - Native ORC MR          1023 / 1026         15.4          65.1       1.5X
    +        Read partition column - Native ORC Vectorized    54 /   56        292.3           3.4      28.7X
    +        Read partition column - Hive built-in ORC      1285 / 1289         12.2          81.7       1.2X
    +        Read both columns - Native ORC MR              1564 / 1565         10.1          99.4       1.0X
    +        Read both columns - Native ORC Vectorized       336 /  340         46.8          21.4       4.6X
    +        Read both columns - Hive built-in ORC          2100 / 2123          7.5         133.5       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def repeatedStringScanBenchmark(values: Int): Unit = {
    +    val benchmark = new Benchmark("Repeated String", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1"))
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        Repeated String:                         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1338 / 1340          7.8         127.6       1.0X
    +        Native ORC Vectorized                          342 /  350         30.6          32.6       3.9X
    +        Hive built-in ORC                             2036 / 2117          5.2         194.2       0.7X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = {
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        spark.range(values).createOrReplaceTempView("t1")
    +
    +        prepareTable(
    +          dir,
    +          spark.sql(
    +            s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " +
    +            s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1"))
    +
    +        val benchmark = new Benchmark(s"String with Nulls Scan ($fractionOfNulls%)", values)
    +
    +        benchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +              "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +          }
    +        }
    +
    +        benchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        benchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " +
    +            "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        String with Nulls Scan (0.0%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2510 / 2527          4.2         239.3       1.0X
    +        Native ORC Vectorized                         1012 / 1012         10.4          96.5       2.5X
    +        Hive built-in ORC                             3967 / 3984          2.6         378.3       0.6X
    +
    +        String with Nulls Scan (0.5%):           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2374 / 2388          4.4         226.4       1.0X
    +        Native ORC Vectorized                         1269 / 1275          8.3         121.0       1.9X
    +        Hive built-in ORC                             2994 / 2998          3.5         285.5       0.8X
    +
    +        String with Nulls Scan (0.95%):          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1285 / 1312          8.2         122.6       1.0X
    +        Native ORC Vectorized                          500 /  508         21.0          47.7       2.6X
    +        Hive built-in ORC                             1630 / 1638          6.4         155.5       0.8X
    +        */
    +        benchmark.run()
    +      }
    +    }
    +  }
    +
    +  def columnsBenchmark(values: Int, width: Int): Unit = {
    +    val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from $width columns", values)
    +
    +    withTempPath { dir =>
    +      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
    +        import spark.implicits._
    +        val middle = width / 2
    +        val selectExpr = (1 to width).map(i => s"value as c$i")
    +        spark.range(values).map(_ => Random.nextLong).toDF()
    +          .selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
    +
    +        prepareTable(dir, spark.sql("SELECT * FROM t1"))
    +
    +        sqlBenchmark.addCase("Native ORC MR") { _ =>
    +          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
    +            spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +          }
    +        }
    +
    +        sqlBenchmark.addCase("Native ORC Vectorized") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect()
    +        }
    +
    +        sqlBenchmark.addCase("Hive built-in ORC") { _ =>
    +          spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect()
    +        }
    +
    +        /*
    +        Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2
    +        Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz
    +
    +        SQL Single Column Scan from 100 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 1107 / 1111          0.9        1055.4       1.0X
    +        Native ORC Vectorized                           93 /  102         11.3          88.8      11.9X
    +        Hive built-in ORC                              377 /  389          2.8         359.6       2.9X
    +
    +        SQL Single Column Scan from 200 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 2258 / 2270          0.5        2153.8       1.0X
    +        Native ORC Vectorized                          160 /  168          6.6         152.6      14.1X
    +        Hive built-in ORC                              591 /  597          1.8         563.7       3.8X
    +
    +        SQL Single Column Scan from 300 columns: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    +        ------------------------------------------------------------------------------------------------
    +        Native ORC MR                                 3364 / 3391          0.3        3208.3       1.0X
    +        Native ORC Vectorized                          273 /  284          3.8         260.1      12.3X
    +        Hive built-in ORC                              831 /  842          1.3         792.8       4.0X
    --- End diff --
    
    What does Hive ORC use? The MR reader or vectorized reader or something else?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158559282
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    --- End diff --
    
    Yep. I can add that. But, do you think we can proceed this approach?
    Based on the previous advice, I think you prefer @viirya 's approach.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85534 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85534/testReport)** for PR 19943 at commit [`9f225e2`](https://github.com/apache/spark/commit/9f225e23b562c5baa7d137b13c697cdeca5e9d43).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160069485
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    --- End diff --
    
    nit: `Vectorized ORC Row Batch.`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85763 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85763/testReport)** for PR 19943 at commit [`0a44d7d`](https://github.com/apache/spark/commit/0a44d7d20e2a0df71fb499db67e0e4779fa46874).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160077992
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private StructType requiredSchema;
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    --- End diff --
    
    Done.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160077944
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private VectorizedRowBatch batch;
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private int[] requestedColIds;
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private org.apache.orc.RecordReader recordReader;
    +
    +  /**
    +   * Required Schema.
    --- End diff --
    
    Yep. It's removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159222343
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala ---
    @@ -110,4 +107,22 @@ object OrcUtils extends Logging {
           }
         }
       }
    +
    +  /**
    +   * Return a fixed ORC schema with data schema information, if needed.
    +   * The schema inside old ORC files might consist of invalid column names like '_col0'.
    +   */
    +  def getFixedTypeDescription(
    +      schema: TypeDescription,
    +      dataSchema: StructType): TypeDescription = {
    +    if (schema.getFieldNames.asScala.forall(_.startsWith("_col"))) {
    +      var schemaString = schema.toString
    +      dataSchema.zipWithIndex.foreach { case (field: StructField, index: Int) =>
    --- End diff --
    
    shall we make sure `schema` and `dataSchema` have same number of fields?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19943
  
    **[Test build #85818 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85818/testReport)** for PR 19943 at commit [`b623ca4`](https://github.com/apache/spark/commit/b623ca4621d22d294974cb3a7f88260052b1f38c).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r158899707
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendBoolean(data(index) == 1)
    +                index += 1
    +              }
    +
    +            case ByteType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendByte(data(index).toByte)
    +                index += 1
    +              }
    +            case ShortType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    +                toColumn.appendShort(data(index).toShort)
    +                index += 1
    +              }
    +            case IntegerType | DateType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    I see. got it


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r160077922
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc;
    +
    +import java.io.IOException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.RecordReader;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit;
    +import org.apache.orc.OrcConf;
    +import org.apache.orc.OrcFile;
    +import org.apache.orc.Reader;
    +import org.apache.orc.TypeDescription;
    +import org.apache.orc.mapred.OrcInputFormat;
    +import org.apache.orc.storage.common.type.HiveDecimal;
    +import org.apache.orc.storage.ql.exec.vector.*;
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
    +
    +import org.apache.spark.memory.MemoryMode;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
    +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
    +import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
    +import org.apache.spark.sql.types.*;
    +import org.apache.spark.sql.vectorized.ColumnarBatch;
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + * After creating, `initialize` and `setRequiredSchema` should be called sequentially.
    + */
    +public class JavaOrcColumnarBatchReader extends RecordReader<Void, ColumnarBatch> {
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private Reader reader;
    +
    +  /**
    +   * Vectorized Row Batch.
    --- End diff --
    
    Yep.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19943: [SPARK-16060][SQL] Support Vectorized ORC Reader

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19943#discussion_r159221758
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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.spark.sql.execution.datasources.orc
    +
    +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
    +import org.apache.hadoop.mapreduce.lib.input.FileSplit
    +import org.apache.orc._
    +import org.apache.orc.mapred.OrcInputFormat
    +import org.apache.orc.storage.ql.exec.vector._
    +import org.apache.orc.storage.serde2.io.HiveDecimalWritable
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.memory.MemoryMode
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.execution.vectorized._
    +import org.apache.spark.sql.types._
    +
    +
    +/**
    + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch.
    + */
    +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging {
    +  import OrcColumnarBatchReader._
    +
    +  /**
    +   * ORC File Reader.
    +   */
    +  private var reader: Reader = _
    +
    +  /**
    +   * Vectorized Row Batch.
    +   */
    +  private var batch: VectorizedRowBatch = _
    +
    +  /**
    +   * Requested Column IDs.
    +   */
    +  private var requestedColIds: Array[Int] = _
    +
    +  /**
    +   * Record reader from row batch.
    +   */
    +  private var rows: org.apache.orc.RecordReader = _
    +
    +  /**
    +   * Required Schema.
    +   */
    +  private var requiredSchema: StructType = _
    +
    +  /**
    +   * ColumnarBatch for vectorized execution by whole-stage codegen.
    +   */
    +  private var columnarBatch: ColumnarBatch = _
    +
    +  /**
    +   * Writable columnVectors of ColumnarBatch.
    +   */
    +  private var columnVectors: Seq[WritableColumnVector] = _
    +
    +  /**
    +   * The number of rows read and considered to be returned.
    +   */
    +  private var rowsReturned: Long = 0L
    +
    +  /**
    +   * Total number of rows.
    +   */
    +  private var totalRowCount: Long = 0L
    +
    +  override def getCurrentKey: Void = null
    +
    +  override def getCurrentValue: ColumnarBatch = columnarBatch
    +
    +  override def getProgress: Float = rowsReturned.toFloat / totalRowCount
    +
    +  override def nextKeyValue(): Boolean = nextBatch()
    +
    +  override def close(): Unit = {
    +    if (columnarBatch != null) {
    +      columnarBatch.close()
    +      columnarBatch = null
    +    }
    +    if (rows != null) {
    +      rows.close()
    +      rows = null
    +    }
    +  }
    +
    +  /**
    +   * Initialize ORC file reader and batch record reader.
    +   * Please note that `setRequiredSchema` is needed to be called after this.
    +   */
    +  override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = {
    +    val fileSplit = inputSplit.asInstanceOf[FileSplit]
    +    val conf = taskAttemptContext.getConfiguration
    +    reader = OrcFile.createReader(
    +      fileSplit.getPath,
    +      OrcFile.readerOptions(conf)
    +        .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf))
    +        .filesystem(fileSplit.getPath.getFileSystem(conf)))
    +
    +    val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength)
    +    rows = reader.rows(options)
    +  }
    +
    +  /**
    +   * Set required schema and partition information.
    +   * With this information, this creates ColumnarBatch with the full schema.
    +   */
    +  def setRequiredSchema(
    +      orcSchema: TypeDescription,
    +      requestedColIds: Array[Int],
    +      resultSchema: StructType,
    +      requiredSchema: StructType,
    +      partitionValues: InternalRow): Unit = {
    +    batch = orcSchema.createRowBatch(DEFAULT_SIZE)
    +    assert(!batch.selectedInUse)
    +    totalRowCount = reader.getNumberOfRows
    +    logDebug(s"totalRowCount = $totalRowCount")
    +
    +    this.requiredSchema = requiredSchema
    +    this.requestedColIds = requestedColIds
    +
    +    val memMode = DEFAULT_MEMORY_MODE
    +    val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE
    +    if (memMode == MemoryMode.OFF_HEAP) {
    +      columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    } else {
    +      columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema)
    +    }
    +    columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity)
    +
    +    if (partitionValues.numFields > 0) {
    +      val partitionIdx = requiredSchema.fields.length
    +      for (i <- 0 until partitionValues.numFields) {
    +        ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i)
    +        columnVectors(i + partitionIdx).setIsConstant()
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Return true if there exists more data in the next batch. If exists, prepare the next batch
    +   * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns.
    +   */
    +  private def nextBatch(): Boolean = {
    +    if (rowsReturned >= totalRowCount) {
    +      return false
    +    }
    +
    +    rows.nextBatch(batch)
    +    val batchSize = batch.size
    +    if (batchSize == 0) {
    +      return false
    +    }
    +    rowsReturned += batchSize
    +    columnarBatch.reset()
    +    columnarBatch.setNumRows(batchSize)
    +
    +    var i = 0
    +    while (i < requiredSchema.length) {
    +      val field = requiredSchema(i)
    +      val toColumn = columnVectors(i)
    +
    +      if (requestedColIds(i) < 0) {
    +        toColumn.appendNulls(batchSize)
    +      } else {
    +        val fromColumn = batch.cols(requestedColIds(i))
    +
    +        if (fromColumn.isRepeating) {
    +          if (fromColumn.isNull(0)) {
    +            toColumn.appendNulls(batchSize)
    +          } else {
    +            field.dataType match {
    +              case BooleanType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1
    +                toColumn.appendBooleans(batchSize, data)
    +
    +              case ByteType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte
    +                toColumn.appendBytes(batchSize, data)
    +              case ShortType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort
    +                toColumn.appendShorts(batchSize, data)
    +              case IntegerType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +              case LongType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0)
    +                toColumn.appendLongs(batchSize, data)
    +
    +              case DateType =>
    +                val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt
    +                toColumn.appendInts(batchSize, data)
    +
    +              case TimestampType =>
    +                val data = fromColumn.asInstanceOf[TimestampColumnVector]
    +                toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0))
    +
    +              case FloatType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat
    +                toColumn.appendFloats(batchSize, data)
    +              case DoubleType =>
    +                val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0)
    +                toColumn.appendDoubles(batchSize, data)
    +
    +              case StringType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +              case BinaryType =>
    +                val data = fromColumn.asInstanceOf[BytesColumnVector]
    +                var index = 0
    +                while (index < batchSize) {
    +                  toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0))
    +                  index += 1
    +                }
    +
    +              case DecimalType.Fixed(precision, scale) =>
    +                val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0)
    +                appendDecimalWritable(toColumn, precision, scale, d)
    +
    +              case dt =>
    +                throw new UnsupportedOperationException(s"Unsupported Data Type: $dt")
    +            }
    +          }
    +        } else if (!field.nullable || fromColumn.noNulls) {
    +          field.dataType match {
    +            case BooleanType =>
    +              val data = fromColumn.asInstanceOf[LongColumnVector].vector
    +              var index = 0
    +              while (index < batchSize) {
    --- End diff --
    
    hmm, so the orc batch always return long as value? that seems very inefficient...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org