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

Update getFileScanRDD shim for recent changes in Spark 3.3.0 [databricks] #4427

Merged
merged 2 commits into from
Dec 23, 2021
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,22 @@ package com.nvidia.spark.rapids.shims.spark320
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.v2._

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
import org.apache.spark.sql.types.StructType

class Spark320Shims extends Spark32XShims with Spark30Xuntil33XShims {
override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION

override def getFileScanRDD(
sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = {
new FileScanRDD(sparkSession, readFunction, filePartitions)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,22 @@ package com.nvidia.spark.rapids.shims.spark321
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.v2._

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
import org.apache.spark.sql.types.StructType

class Spark321Shims extends Spark32XShims with Spark30Xuntil33XShims {
override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION

override def getFileScanRDD(
sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = {
new FileScanRDD(sparkSession, readFunction, filePartitions)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -508,7 +508,9 @@ abstract class Spark30XdbShims extends Spark30XdbShimsBase with Logging {
override def getFileScanRDD(
sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition]): RDD[InternalRow] = {
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = {
new GpuFileScanRDD(sparkSession, readFunction, filePartitions)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,9 @@ trait Spark301until320Shims extends SparkShims {
override def getFileScanRDD(
sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition]): RDD[InternalRow] = {
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = {
new FileScanRDD(sparkSession, readFunction, filePartitions)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -627,7 +627,9 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging {
override def getFileScanRDD(
sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition]): RDD[InternalRow] = {
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = {
new GpuFileScanRDD(sparkSession, readFunction, filePartitions)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@ import org.apache.parquet.schema.MessageType
import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
import org.apache.spark.rapids.shims.v2.GpuShuffleExchangeExec
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.Resolver
Expand Down Expand Up @@ -854,13 +853,6 @@ trait Spark32XShims extends SparkShims with Logging {
}
}

override def getFileScanRDD(
sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition]): RDD[InternalRow] = {
new FileScanRDD(sparkSession, readFunction, filePartitions)
}

override def createFilePartition(index: Int, files: Array[PartitionedFile]): FilePartition = {
FilePartition(index, files)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,14 @@ package com.nvidia.spark.rapids.shims.v2

import com.nvidia.spark.rapids._

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.csv.CSVOptions
import org.apache.spark.sql.execution._
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
import org.apache.spark.sql.types.StructType

trait Spark33XShims extends Spark32XShims {
override def neverReplaceShowCurrentNamespaceCommand: ExecRule[_ <: SparkPlan] = null
Expand All @@ -31,4 +37,13 @@ trait Spark33XShims extends Spark32XShims {
override def timestampFormatInRead(csvOpts: CSVOptions): Option[String] = {
csvOpts.timestampFormatInRead
}

override def getFileScanRDD(
sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = {
new FileScanRDD(sparkSession, readFunction, filePartitions, readDataSchema, metadataColumns)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog}
import org.apache.spark.sql.catalyst.csv.CSVOptions
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, ExprId, NullOrdering, SortDirection, SortOrder}
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExprId, NullOrdering, SortDirection, SortOrder}
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning}
import org.apache.spark.sql.catalyst.trees.TreeNode
Expand Down Expand Up @@ -170,9 +170,11 @@ trait SparkShims {
maxSplitBytes: Long,
relation: HadoopFsRelation): Array[PartitionedFile]
def getFileScanRDD(
sparkSession: SparkSession,
readFunction: (PartitionedFile) => Iterator[InternalRow],
filePartitions: Seq[FilePartition]): RDD[InternalRow]
sparkSession: SparkSession,
readFunction: (PartitionedFile) => Iterator[InternalRow],
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference] = Seq.empty): RDD[InternalRow]

def getFileSourceMaxMetadataValueLength(sqlConf: SQLConf): Int

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -497,7 +497,8 @@ case class GpuFileSourceScanExec(

if (isPerFileReadEnabled) {
logInfo("Using the original per file parquet reader")
ShimLoader.getSparkShims.getFileScanRDD(fsRelation.sparkSession, readFile.get, partitions)
ShimLoader.getSparkShims.getFileScanRDD(fsRelation.sparkSession, readFile.get, partitions,
requiredSchema)
} else {
// here we are making an optimization to read more then 1 file at a time on the CPU side
// if they are small files before sending it down to the GPU
Expand Down