Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is weird, where do we change org.apache.spark.sql.vectorized.ColumnVector in this PR?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan - yeah it's weird. We don't change ColumnVector class at all. Do you have any idea for how to debug on this? I am still checking why, thanks.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe it's some bugs in Mima, not a bit deal as we know this PR doesn't break binary compatibility.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan - spent some time checking, but still not sure where the issue is, so I agree with you that might be some bug in Mima.

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 @@ -838,6 +838,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 @@ -3328,6 +3335,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