Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

WIP: feat: Add support for Spark 3.5 #555

Closed
wants to merge 13 commits into from
26 changes: 25 additions & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,7 @@ under the License.
<additional.3_4.test.source>spark-3.4-plus</additional.3_4.test.source>
<shims.majorVerSrc>spark-3.x</shims.majorVerSrc>
<shims.minorVerSrc>spark-3.4</shims.minorVerSrc>
<shims.extraVerSrc>spark-pre-3.5</shims.extraVerSrc>
</properties>

<dependencyManagement>
Expand Down Expand Up @@ -527,7 +528,9 @@ under the License.
<!-- we don't add special test suits for spark-3.2, so a not existed dir is specified-->
<additional.3_3.test.source>not-needed-yet</additional.3_3.test.source>
<additional.3_4.test.source>not-needed-yet</additional.3_4.test.source>
<shims.majorVerSrc>spark-3.x</shims.majorVerSrc>
<shims.minorVerSrc>spark-3.2</shims.minorVerSrc>
<shims.extraVerSrc>spark-pre-3.5</shims.extraVerSrc>
</properties>
</profile>

Expand All @@ -539,7 +542,9 @@ under the License.
<spark.version.short>3.3</spark.version.short>
<parquet.version>1.12.0</parquet.version>
<additional.3_4.test.source>not-needed-yet</additional.3_4.test.source>
<shims.majorVerSrc>spark-3.x</shims.majorVerSrc>
<shims.minorVerSrc>spark-3.3</shims.minorVerSrc>
<shims.extraVerSrc>spark-pre-3.5</shims.extraVerSrc>
</properties>
</profile>

Expand All @@ -549,6 +554,25 @@ under the License.
<scala.version>2.12.17</scala.version>
<spark.version.short>3.4</spark.version.short>
<parquet.version>1.13.1</parquet.version>
<shims.majorVerSrc>spark-3.x</shims.majorVerSrc>
<shims.minorVerSrc>spark-3.4</shims.minorVerSrc>
<shims.extraVerSrc>spark-pre-3.5</shims.extraVerSrc>
</properties>
</profile>

<profile>
<!-- FIXME: this is WIP. Tests may fail -->
<id>spark-3.5</id>
<properties>
<scala.version>2.12.17</scala.version>
<spark.version>3.5.1</spark.version>
<spark.version.short>3.5</spark.version.short>
<parquet.version>1.13.1</parquet.version>
<!-- <additional.3_3.test.source>spark-3.3-plus</additional.3_3.test.source>-->
<!-- <additional.3_5.test.source>spark-3.5</additional.3_5.test.source>-->
<shims.majorVerSrc>spark-3.x</shims.majorVerSrc>
<shims.minorVerSrc>spark-3.5</shims.minorVerSrc>
<shims.extraVerSrc>not-needed-yet</shims.extraVerSrc>
</properties>
</profile>

Expand All @@ -564,6 +588,7 @@ under the License.
<parquet.version>1.13.1</parquet.version>
<shims.majorVerSrc>spark-4.0</shims.majorVerSrc>
<shims.minorVerSrc>not-needed-yet</shims.minorVerSrc>
<shims.extraVerSrc>not-needed-yet</shims.extraVerSrc>
<!-- Use jdk17 by default -->
<java.version>17</java.version>
<maven.compiler.source>${java.version}</maven.compiler.source>
Expand Down Expand Up @@ -693,7 +718,6 @@ under the License.
</execution>
</executions>
<configuration>
-->
<scalaVersion>${scala.version}</scalaVersion>
<checkMultipleScalaVersions>true</checkMultipleScalaVersions>
<failOnMultipleScalaVersions>true</failOnMultipleScalaVersions>
Expand Down
2 changes: 2 additions & 0 deletions spark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -254,6 +254,7 @@ under the License.
<source>src/test/${additional.3_4.test.source}</source>
<source>src/test/${shims.majorVerSrc}</source>
<source>src/test/${shims.minorVerSrc}</source>
<source>src/test/${shims.extraVerSrc}</source>
</sources>
</configuration>
</execution>
Expand All @@ -267,6 +268,7 @@ under the License.
<sources>
<source>src/main/${shims.majorVerSrc}</source>
<source>src/main/${shims.minorVerSrc}</source>
<source>src/main/${shims.extraVerSrc}</source>
</sources>
</configuration>
</execution>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,10 +41,11 @@ import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.comet.{CometExec, CometMetricNode, CometPlan}
import org.apache.spark.sql.comet.shims.ShimCometShuffleWriteProcessor
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeExec, ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructField
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util.MutablePair
import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator}
Expand Down Expand Up @@ -390,7 +391,8 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec {
val pageSize = SparkEnv.get.memoryManager.pageSizeBytes

val sorter = UnsafeExternalRowSorter.createWithRecordComparator(
fromAttributes(outputAttributes),
StructType(
outputAttributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))),
recordComparatorSupplier,
prefixComparator,
prefixComputer,
Expand Down Expand Up @@ -434,8 +436,8 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec {
serializer,
shuffleWriterProcessor = ShuffleExchangeExec.createShuffleWriteProcessor(writeMetrics),
shuffleType = CometColumnarShuffle,
schema = Some(fromAttributes(outputAttributes)))

schema = Some(StructType(outputAttributes.map(a =>
StructField(a.name, a.dataType, a.nullable, a.metadata)))))
dependency
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,231 @@
/*
* 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.comet.parquet

import scala.collection.JavaConverters

import org.apache.hadoop.conf.Configuration
import org.apache.parquet.filter2.predicate.FilterApi
import org.apache.parquet.hadoop.ParquetInputFormat
import org.apache.parquet.hadoop.metadata.FileMetaData
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec
import org.apache.spark.sql.execution.datasources.DataSourceUtils
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.execution.datasources.RecordReaderIterator
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions
import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.LegacyBehaviorPolicy
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.{DateType, StructType, TimestampType}
import org.apache.spark.util.SerializableConfiguration

import org.apache.comet.CometConf
import org.apache.comet.MetricsSupport
import org.apache.comet.shims.ShimSQLConf
import org.apache.comet.vector.CometVector

/**
* A Comet specific Parquet format. This mostly reuse the functionalities from Spark's
* [[ParquetFileFormat]], but overrides:
*
* - `vectorTypes`, so Spark allocates [[CometVector]] instead of it's own on-heap or off-heap
* column vector in the whole-stage codegen path.
* - `supportBatch`, which simply returns true since data types should have already been checked
* in [[org.apache.comet.CometSparkSessionExtensions]]
* - `buildReaderWithPartitionValues`, so Spark calls Comet's Parquet reader to read values.
*/
class CometParquetFileFormat extends ParquetFileFormat with MetricsSupport with ShimSQLConf {
override def shortName(): String = "parquet"
override def toString: String = "CometParquet"
override def hashCode(): Int = getClass.hashCode()
override def equals(other: Any): Boolean = other.isInstanceOf[CometParquetFileFormat]

override def vectorTypes(
requiredSchema: StructType,
partitionSchema: StructType,
sqlConf: SQLConf): Option[Seq[String]] = {
val length = requiredSchema.fields.length + partitionSchema.fields.length
Option(Seq.fill(length)(classOf[CometVector].getName))
}

override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = true

override def buildReaderWithPartitionValues(
sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val sqlConf = sparkSession.sessionState.conf
CometParquetFileFormat.populateConf(sqlConf, hadoopConf)
val broadcastedHadoopConf =
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))

val isCaseSensitive = sqlConf.caseSensitiveAnalysis
val useFieldId = CometParquetUtils.readFieldId(sqlConf)
val ignoreMissingIds = CometParquetUtils.ignoreMissingIds(sqlConf)
val pushDownDate = sqlConf.parquetFilterPushDownDate
val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
val pushDownStringPredicate = getPushDownStringPredicate(sqlConf)
val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
val optionsMap = CaseInsensitiveMap[String](options)
val parquetOptions = new ParquetOptions(optionsMap, sqlConf)
val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead
val parquetFilterPushDown = sqlConf.parquetFilterPushDown

// Comet specific configurations
val capacity = CometConf.COMET_BATCH_SIZE.get(sqlConf)

(file: PartitionedFile) => {
val sharedConf = broadcastedHadoopConf.value.value
val footer = FooterReader.readFooter(sharedConf, file)
val footerFileMetaData = footer.getFileMetaData
val datetimeRebaseSpec = CometParquetFileFormat.getDatetimeRebaseSpec(
file,
requiredSchema,
sharedConf,
footerFileMetaData,
datetimeRebaseModeInRead)

val pushed = if (parquetFilterPushDown) {
val parquetSchema = footerFileMetaData.getSchema
val parquetFilters = new ParquetFilters(
parquetSchema,
pushDownDate,
pushDownTimestamp,
pushDownDecimal,
pushDownStringPredicate,
pushDownInFilterThreshold,
isCaseSensitive,
datetimeRebaseSpec)
filters
// Collects all converted Parquet filter predicates. Notice that not all predicates can
// be converted (`ParquetFilters.createFilter` returns an `Option`). That's why a
// `flatMap` is used here.
.flatMap(parquetFilters.createFilter)
.reduceOption(FilterApi.and)
} else {
None
}
pushed.foreach(p => ParquetInputFormat.setFilterPredicate(sharedConf, p))

val batchReader = new BatchReader(
sharedConf,
file,
footer,
capacity,
requiredSchema,
isCaseSensitive,
useFieldId,
ignoreMissingIds,
datetimeRebaseSpec.mode == LegacyBehaviorPolicy.CORRECTED,
partitionSchema,
file.partitionValues,
JavaConverters.mapAsJavaMap(metrics))
val iter = new RecordReaderIterator(batchReader)
try {
batchReader.init()
iter.asInstanceOf[Iterator[InternalRow]]
} catch {
case e: Throwable =>
iter.close()
throw e
}
}
}
}

object CometParquetFileFormat extends Logging {

/**
* Populates Parquet related configurations from the input `sqlConf` to the `hadoopConf`
*/
def populateConf(sqlConf: SQLConf, hadoopConf: Configuration): Unit = {
hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone)
hadoopConf.setBoolean(
SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
sqlConf.nestedSchemaPruningEnabled)
hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key, sqlConf.caseSensitiveAnalysis)

// Sets flags for `ParquetToSparkSchemaConverter`
hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, sqlConf.isParquetBinaryAsString)
hadoopConf.setBoolean(
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
sqlConf.isParquetINT96AsTimestamp)

// Comet specific configs
hadoopConf.setBoolean(
CometConf.COMET_PARQUET_ENABLE_DIRECT_BUFFER.key,
CometConf.COMET_PARQUET_ENABLE_DIRECT_BUFFER.get())
hadoopConf.setBoolean(
CometConf.COMET_USE_DECIMAL_128.key,
CometConf.COMET_USE_DECIMAL_128.get())
hadoopConf.setBoolean(
CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key,
CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.get())
}

def getDatetimeRebaseSpec(
file: PartitionedFile,
sparkSchema: StructType,
sharedConf: Configuration,
footerFileMetaData: FileMetaData,
datetimeRebaseModeInRead: String): RebaseSpec = {
val exceptionOnRebase = sharedConf.getBoolean(
CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key,
CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.defaultValue.get)
var datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(
footerFileMetaData.getKeyValueMetaData.get,
datetimeRebaseModeInRead)
val hasDateOrTimestamp = sparkSchema.exists(f =>
f.dataType match {
case DateType | TimestampType => true
case _ => false
})

if (hasDateOrTimestamp && datetimeRebaseSpec.mode == LegacyBehaviorPolicy.LEGACY) {
if (exceptionOnRebase) {
logWarning(
s"""Found Parquet file $file that could potentially contain dates/timestamps that were
written in legacy hybrid Julian/Gregorian calendar. Unlike Spark 3+, which will rebase
and return these according to the new Proleptic Gregorian calendar, Comet will throw
exception when reading them. If you want to read them as it is according to the hybrid
Julian/Gregorian calendar, please set `spark.comet.exceptionOnDatetimeRebase` to
false. Otherwise, if you want to read them according to the new Proleptic Gregorian
calendar, please disable Comet for this query.""")
} else {
// do not throw exception on rebase - read as it is
datetimeRebaseSpec = datetimeRebaseSpec.copy(LegacyBehaviorPolicy.CORRECTED)
}
}

datetimeRebaseSpec
}
}
Loading
Loading