@@ -18,12 +18,14 @@ package org.apache.spark.sql.blaze
1818import java .io .File
1919import java .util .UUID
2020import org .apache .commons .lang3 .reflect .FieldUtils
21+ import org .apache .hadoop .fs .Path
2122import org .apache .spark .OneToOneDependency
2223import org .apache .spark .ShuffleDependency
2324import org .apache .spark .SparkEnv
2425import org .apache .spark .SparkException
2526import org .apache .spark .TaskContext
2627import org .apache .spark .internal .Logging
28+ import org .apache .spark .paths .SparkPath
2729import org .apache .spark .rdd .RDD
2830import org .apache .spark .scheduler .MapStatus
2931import org .apache .spark .shuffle .IndexShuffleBlockResolver
@@ -33,6 +35,7 @@ import org.apache.spark.sql.SQLContext
3335import org .apache .spark .sql .SparkSession
3436import org .apache .spark .sql .blaze .BlazeConverters .ForceNativeExecutionWrapperBase
3537import org .apache .spark .sql .blaze .NativeConverters .NativeExprWrapperBase
38+ import org .apache .spark .sql .catalyst .InternalRow
3639import org .apache .spark .sql .catalyst .catalog .CatalogTable
3740import org .apache .spark .sql .catalyst .expressions .Attribute
3841import org .apache .spark .sql .catalyst .expressions .Expression
@@ -97,6 +100,7 @@ import org.apache.spark.sql.execution.blaze.plan._
97100import org .apache .spark .sql .execution .blaze .shuffle .RssPartitionWriterBase
98101import org .apache .spark .sql .execution .blaze .shuffle .celeborn .BlazeCelebornShuffleManager
99102import org .apache .spark .sql .execution .blaze .shuffle .BlazeBlockStoreShuffleReaderBase
103+ import org .apache .spark .sql .execution .datasources .PartitionedFile
100104import org .apache .spark .sql .execution .exchange .{BroadcastExchangeLike , ReusedExchangeExec }
101105import org .apache .spark .sql .execution .joins .blaze .plan .NativeBroadcastJoinExec
102106import org .apache .spark .sql .execution .joins .blaze .plan .NativeShuffledHashJoinExecProvider
@@ -818,6 +822,35 @@ class ShimsImpl extends Shims with Logging {
818822 NativeExprWrapper (nativeExpr, dataType, nullable)
819823 }
820824
825+ @ enableIf(
826+ Seq (" spark-3.0" , " spark-3.1" , " spark-3.2" , " spark-3.3" ).contains(
827+ System .getProperty(" blaze.shim" )))
828+ override def getPartitionedFile (
829+ partitionValues : InternalRow ,
830+ filePath : String ,
831+ offset : Long ,
832+ size : Long ): PartitionedFile =
833+ PartitionedFile (partitionValues, filePath, offset, size)
834+
835+ @ enableIf(Seq (" spark-3.4" , " spark-3.5" ).contains(System .getProperty(" blaze.shim" )))
836+ override def getPartitionedFile (
837+ partitionValues : InternalRow ,
838+ filePath : String ,
839+ offset : Long ,
840+ size : Long ): PartitionedFile =
841+ PartitionedFile (partitionValues, SparkPath .fromPath(new Path (filePath)), offset, size)
842+
843+ @ enableIf(
844+ Seq (" spark-3.1" , " spark-3.2" , " spark-3.3" , " spark-3.4" , " spark-3.5" ).contains(
845+ System .getProperty(" blaze.shim" )))
846+ override def getMinPartitionNum (sparkSession : SparkSession ): Int =
847+ sparkSession.sessionState.conf.filesMinPartitionNum
848+ .getOrElse(sparkSession.sparkContext.defaultParallelism)
849+
850+ @ enableIf(Seq (" spark-3.0" ).contains(System .getProperty(" blaze.shim" )))
851+ override def getMinPartitionNum (sparkSession : SparkSession ): Int =
852+ sparkSession.sparkContext.defaultParallelism
853+
821854 @ enableIf(
822855 Seq (" spark-3.0" , " spark-3.1" , " spark-3.2" , " spark-3.3" ).contains(
823856 System .getProperty(" blaze.shim" )))
0 commit comments