-
Notifications
You must be signed in to change notification settings - Fork 242
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Spark 4.0] Account for
PartitionedFileUtil.splitFiles
signature ch…
…ange. (#10857) * Account for PartitionedFileUtil.splitFiles signature change. Fixes #10299. In Apache Spark 4.0, the signature of `PartitionedFileUtil.splitFiles` was changed to remove unused parameters (apache/spark@eabea643c74). This causes the Spark RAPIDS plugin build to break with Spark 4.0. This commit introduces a shim to account for the signature change. Signed-off-by: MithunR <[email protected]> * Common base for PartitionFileUtilsShims. Signed-off-by: MithunR <[email protected]> * Reusing existing PartitionedFileUtilsShims. * More refactor, for pre-3.5 compile. * Updated Copyright date. * Fixed style error. * Re-fixed the copyright year. * Added missing import. --------- Signed-off-by: MithunR <[email protected]>
- Loading branch information
Showing
6 changed files
with
141 additions
and
26 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
45 changes: 45 additions & 0 deletions
45
.../src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
/* | ||
* Copyright (c) 2024, NVIDIA CORPORATION. | ||
* | ||
* Licensed 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. | ||
*/ | ||
|
||
/*** spark-rapids-shim-json-lines | ||
{"spark": "340"} | ||
{"spark": "341"} | ||
{"spark": "342"} | ||
{"spark": "343"} | ||
{"spark": "350"} | ||
{"spark": "351"} | ||
{"spark": "400"} | ||
spark-rapids-shim-json-lines ***/ | ||
package com.nvidia.spark.rapids.shims | ||
|
||
import org.apache.spark.paths.SparkPath | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.execution.datasources.PartitionedFile | ||
|
||
trait PartitionedFileUtilsShimBase { | ||
|
||
// Wrapper for case class constructor so Java code can access | ||
// the default values across Spark versions. | ||
def newPartitionedFile(partitionValues: InternalRow, | ||
filePath: String, | ||
start: Long, | ||
length: Long): PartitionedFile = PartitionedFile(partitionValues, | ||
SparkPath.fromPathString(filePath), start, length) | ||
|
||
def withNewLocations(pf: PartitionedFile, locations: Seq[String]): PartitionedFile = { | ||
pf.copy(locations = locations.toArray) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
38 changes: 38 additions & 0 deletions
38
...ugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,38 @@ | ||
/* | ||
* Copyright (c) 2024, NVIDIA CORPORATION. | ||
* | ||
* Licensed 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. | ||
*/ | ||
|
||
/*** spark-rapids-shim-json-lines | ||
{"spark": "350"} | ||
{"spark": "351"} | ||
spark-rapids-shim-json-lines ***/ | ||
package com.nvidia.spark.rapids.shims | ||
|
||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.execution.PartitionedFileUtil | ||
import org.apache.spark.sql.execution.datasources.{FileStatusWithMetadata, PartitionedFile} | ||
|
||
object PartitionedFileUtilsShim extends PartitionedFileUtilsShimBase { | ||
// In Spark 4.0, PartitionedFileUtil.splitFiles lost its `sparkSession` parameter. | ||
// This pre-Spark-4.0 shim keeps the `sparkSession` parameter. | ||
def splitFiles(sparkSession: SparkSession, | ||
file: FileStatusWithMetadata, | ||
isSplitable: Boolean, | ||
maxSplitBytes: Long, | ||
partitionValues: InternalRow): Seq[PartitionedFile] = { | ||
PartitionedFileUtil.splitFiles(sparkSession, file, isSplitable, maxSplitBytes, partitionValues) | ||
} | ||
} |
40 changes: 40 additions & 0 deletions
40
...ugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
* Copyright (c) 2024, NVIDIA CORPORATION. | ||
* | ||
* Licensed 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. | ||
*/ | ||
|
||
/*** spark-rapids-shim-json-lines | ||
{"spark": "400"} | ||
spark-rapids-shim-json-lines ***/ | ||
package com.nvidia.spark.rapids.shims | ||
|
||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.execution.PartitionedFileUtil | ||
import org.apache.spark.sql.execution.datasources.{FileStatusWithMetadata, PartitionedFile} | ||
|
||
object PartitionedFileUtilsShim extends PartitionedFileUtilsShimBase { | ||
|
||
// In Spark 4.0, PartitionedFileUtil.splitFiles lost its `sparkSession` parameter. | ||
// This Spark-4.0+ shim ignores the `sparkSession` parameter. | ||
def splitFiles(sparkSession: SparkSession, | ||
file: FileStatusWithMetadata, | ||
isSplitable: Boolean, | ||
maxSplitBytes: Long, | ||
partitionValues: InternalRow): Seq[PartitionedFile] = { | ||
PartitionedFileUtil.splitFiles(file, isSplitable, maxSplitBytes, partitionValues) | ||
} | ||
|
||
} // object PartitionFileUtilsShim; |