Skip to content

Commit

Permalink
[SPARK-34862][SQL] Support nested column in ORC vectorized reader
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR is to support nested column type in Spark ORC vectorized reader. Currently ORC vectorized reader [does not support nested column type (struct, array and map)](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala#L138). We implemented nested column vectorized reader for FB-ORC in our internal fork of Spark. We are seeing performance improvement compared to non-vectorized reader when reading nested columns. In addition, this can also help improve the non-nested column performance when reading non-nested and nested columns together in one query.

Before this PR:

* `OrcColumnVector` is the implementation class for Spark's `ColumnVector` to wrap Hive's/ORC's `ColumnVector` to read `AtomicType` data.

After this PR:

* `OrcColumnVector` is an abstract class to keep interface being shared between multiple implementation class of orc column vectors, namely `OrcAtomicColumnVector` (for `AtomicType`), `OrcArrayColumnVector` (for `ArrayType`), `OrcMapColumnVector` (for `MapType`), `OrcStructColumnVector` (for `StructType`). So the original logic to read `AtomicType` data is moved from `OrcColumnVector` to `OrcAtomicColumnVector`. The abstract class of `OrcColumnVector` is needed here because of supporting nested column (i.e. nested column vectors).
* A utility method `OrcColumnVectorUtils.toOrcColumnVector` is added to create Spark's `OrcColumnVector` from Hive's/ORC's `ColumnVector`.
* A new user-facing config `spark.sql.orc.enableNestedColumnVectorizedReader` is added to control enabling/disabling vectorized reader for nested columns. The default value is false (i.e. disabling by default). For certain tables having deep nested columns, vectorized reader might take too much memory for each sub-column vectors, compared to non-vectorized reader. So providing a config here to work around OOM for query reading wide and deep nested columns if any. We plan to enable it by default on 3.3. Leave it disable in 3.2 in case for any unknown bugs.

### Why are the changes needed?

Improve query performance when reading nested columns from ORC file format.
Tested with locally adding a small benchmark in `OrcReadBenchmark.scala`. Seeing more than 1x run time improvement.

```
Running benchmark: SQL Nested Column Scan
  Running case: Native ORC MR
  Stopped after 2 iterations, 37850 ms
  Running case: Native ORC Vectorized (Enabled Nested Column)
  Stopped after 2 iterations, 15892 ms
  Running case: Native ORC Vectorized (Disabled Nested Column)
  Stopped after 2 iterations, 37954 ms
  Running case: Hive built-in ORC
  Stopped after 2 iterations, 35118 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
SQL Nested Column Scan:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------------
Native ORC MR                                           18706          18925         310          0.1       17839.6       1.0X
Native ORC Vectorized (Enabled Nested Column)            7625           7946         455          0.1        7271.6       2.5X
Native ORC Vectorized (Disabled Nested Column)          18415          18977         796          0.1       17561.5       1.0X
Hive built-in ORC                                       17469          17559         127          0.1       16660.1       1.1X
```

Benchmark:

```
nestedColumnScanBenchmark(1024 * 1024)
def nestedColumnScanBenchmark(values: Int): Unit = {
    val benchmark = new Benchmark(s"SQL Nested Column Scan", values, output = output)

    withTempPath { dir =>
      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
        import spark.implicits._
        spark.range(values).map(_ => Random.nextLong).map { x =>
          val arrayOfStructColumn = (0 until 5).map(i => (x + i, s"$x" * 5))
          val mapOfStructColumn = Map(
            s"$x" -> (x * 0.1, (x, s"$x" * 100)),
            (s"$x" * 2) -> (x * 0.2, (x, s"$x" * 200)),
            (s"$x" * 3) -> (x * 0.3, (x, s"$x" * 300)))
          (arrayOfStructColumn, mapOfStructColumn)
        }.toDF("col1", "col2")
          .createOrReplaceTempView("t1")

        prepareTable(dir, spark.sql(s"SELECT * FROM t1"))

        benchmark.addCase("Native ORC MR") { _ =>
          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
            spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
          }
        }

        benchmark.addCase("Native ORC Vectorized (Enabled Nested Column)") { _ =>
          spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
        }

        benchmark.addCase("Native ORC Vectorized (Disabled Nested Column)") { _ =>
          withSQLConf(SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "false") {
            spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
          }
        }

        benchmark.addCase("Hive built-in ORC") { _ =>
          spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM hiveOrcTable").noop()
        }

        benchmark.run()
      }
    }
  }
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added one simple test in `OrcSourceSuite.scala` to verify correctness.
Definitely need more unit tests and add benchmark here, but I want to first collect feedback before crafting more tests.

Closes #31958 from c21/orc-vector.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
  • Loading branch information
c21 authored and viirya committed Apr 2, 2021
1 parent f03c7c0 commit 1fc66f6
Show file tree
Hide file tree
Showing 11 changed files with 657 additions and 141 deletions.
17 changes: 16 additions & 1 deletion project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,22 @@ object MimaExcludes {
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.connector.write.V1WriteBuilder"),

// [SPARK-33955] Add latest offsets to source progress
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceProgress.this")
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceProgress.this"),

// [SPARK-34862][SQL] Support nested column in ORC vectorized reader
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getBoolean"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getByte"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getShort"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getInt"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getLong"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getFloat"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getDouble"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getDecimal"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getUTF8String"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getBinary"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getArray"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getMap"),
ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.vectorized.ColumnVector.getChild")
)

// Exclude rules for 3.1.x
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -839,6 +839,13 @@ object SQLConf {
.intConf
.createWithDefault(4096)

val ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED =
buildConf("spark.sql.orc.enableNestedColumnVectorizedReader")
.doc("Enables vectorized orc decoding for nested column.")
.version("3.2.0")
.booleanConf
.createWithDefault(false)

val ORC_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.orc.filterPushdown")
.doc("When true, enable filter pushdown for ORC files.")
.version("1.4.0")
Expand Down Expand Up @@ -3339,6 +3346,9 @@ class SQLConf extends Serializable with Logging {

def orcVectorizedReaderBatchSize: Int = getConf(ORC_VECTORIZED_READER_BATCH_SIZE)

def orcVectorizedReaderNestedColumnEnabled: Boolean =
getConf(ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED)

def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION)

def parquetVectorizedReaderEnabled: Boolean = getConf(PARQUET_VECTORIZED_READER_ENABLED)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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.hive.ql.exec.vector.ColumnVector;

import org.apache.spark.sql.types.ArrayType;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
import org.apache.spark.sql.vectorized.ColumnarArray;
import org.apache.spark.sql.vectorized.ColumnarMap;
import org.apache.spark.unsafe.types.UTF8String;

/**
* A column vector implementation for Spark's {@link ArrayType}.
*/
public class OrcArrayColumnVector extends OrcColumnVector {
private final OrcColumnVector data;
private final long[] offsets;
private final long[] lengths;

OrcArrayColumnVector(
DataType type,
ColumnVector vector,
OrcColumnVector data,
long[] offsets,
long[] lengths) {

super(type, vector);

this.data = data;
this.offsets = offsets;
this.lengths = lengths;
}

@Override
public ColumnarArray getArray(int rowId) {
return new ColumnarArray(data, (int) offsets[rowId], (int) lengths[rowId]);
}

@Override
public boolean getBoolean(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public byte getByte(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public short getShort(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public int getInt(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public long getLong(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public float getFloat(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public double getDouble(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public Decimal getDecimal(int rowId, int precision, int scale) {
throw new UnsupportedOperationException();
}

@Override
public UTF8String getUTF8String(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public byte[] getBinary(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public ColumnarMap getMap(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public org.apache.spark.sql.vectorized.ColumnVector getChild(int ordinal) {
throw new UnsupportedOperationException();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,161 @@
/*
* 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.math.BigDecimal;

import org.apache.hadoop.hive.ql.exec.vector.*;

import org.apache.spark.sql.catalyst.util.DateTimeUtils;
import org.apache.spark.sql.catalyst.util.RebaseDateTime;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.DateType;
import org.apache.spark.sql.types.Decimal;
import org.apache.spark.sql.types.TimestampType;
import org.apache.spark.sql.vectorized.ColumnarArray;
import org.apache.spark.sql.vectorized.ColumnarMap;
import org.apache.spark.unsafe.types.UTF8String;

/**
* A column vector implementation for Spark's AtomicType.
*/
public class OrcAtomicColumnVector extends OrcColumnVector {
private final boolean isTimestamp;
private final boolean isDate;

// Column vector for each type. Only 1 is populated for any type.
private LongColumnVector longData;
private DoubleColumnVector doubleData;
private BytesColumnVector bytesData;
private DecimalColumnVector decimalData;
private TimestampColumnVector timestampData;

OrcAtomicColumnVector(DataType type, ColumnVector vector) {
super(type, vector);

if (type instanceof TimestampType) {
isTimestamp = true;
} else {
isTimestamp = false;
}

if (type instanceof DateType) {
isDate = true;
} else {
isDate = false;
}

if (vector instanceof LongColumnVector) {
longData = (LongColumnVector) vector;
} else if (vector instanceof DoubleColumnVector) {
doubleData = (DoubleColumnVector) vector;
} else if (vector instanceof BytesColumnVector) {
bytesData = (BytesColumnVector) vector;
} else if (vector instanceof DecimalColumnVector) {
decimalData = (DecimalColumnVector) vector;
} else if (vector instanceof TimestampColumnVector) {
timestampData = (TimestampColumnVector) vector;
} else {
throw new UnsupportedOperationException();
}
}

@Override
public boolean getBoolean(int rowId) {
return longData.vector[getRowIndex(rowId)] == 1;
}

@Override
public byte getByte(int rowId) {
return (byte) longData.vector[getRowIndex(rowId)];
}

@Override
public short getShort(int rowId) {
return (short) longData.vector[getRowIndex(rowId)];
}

@Override
public int getInt(int rowId) {
int value = (int) longData.vector[getRowIndex(rowId)];
if (isDate) {
return RebaseDateTime.rebaseJulianToGregorianDays(value);
} else {
return value;
}
}

@Override
public long getLong(int rowId) {
int index = getRowIndex(rowId);
if (isTimestamp) {
return DateTimeUtils.fromJavaTimestamp(timestampData.asScratchTimestamp(index));
} else {
return longData.vector[index];
}
}

@Override
public float getFloat(int rowId) {
return (float) doubleData.vector[getRowIndex(rowId)];
}

@Override
public double getDouble(int rowId) {
return doubleData.vector[getRowIndex(rowId)];
}

@Override
public Decimal getDecimal(int rowId, int precision, int scale) {
if (isNullAt(rowId)) return null;
BigDecimal data = decimalData.vector[getRowIndex(rowId)].getHiveDecimal().bigDecimalValue();
return Decimal.apply(data, precision, scale);
}

@Override
public UTF8String getUTF8String(int rowId) {
if (isNullAt(rowId)) return null;
int index = getRowIndex(rowId);
BytesColumnVector col = bytesData;
return UTF8String.fromBytes(col.vector[index], col.start[index], col.length[index]);
}

@Override
public byte[] getBinary(int rowId) {
if (isNullAt(rowId)) return null;
int index = getRowIndex(rowId);
byte[] binary = new byte[bytesData.length[index]];
System.arraycopy(bytesData.vector[index], bytesData.start[index], binary, 0, binary.length);
return binary;
}

@Override
public ColumnarArray getArray(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public ColumnarMap getMap(int rowId) {
throw new UnsupportedOperationException();
}

@Override
public org.apache.spark.sql.vectorized.ColumnVector getChild(int ordinal) {
throw new UnsupportedOperationException();
}
}
Loading

0 comments on commit 1fc66f6

Please sign in to comment.