diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala
index f845b321a4a..aa4c1232eb0 100644
--- a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala
+++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -63,8 +63,7 @@ abstract class GpuDeltaParquetFileFormatBase extends GpuReadParquetFileFormat {
pushedFilters,
fileScan.rapidsConf,
fileScan.allMetrics,
- fileScan.queryUsesInputFile,
- fileScan.alluxioPathsMap)
+ fileScan.queryUsesInputFile)
}
override def buildReaderWithPartitionValuesAndMetrics(
@@ -75,8 +74,7 @@ abstract class GpuDeltaParquetFileFormatBase extends GpuReadParquetFileFormat {
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
super.buildReaderWithPartitionValuesAndMetrics(
sparkSession,
@@ -86,8 +84,7 @@ abstract class GpuDeltaParquetFileFormatBase extends GpuReadParquetFileFormat {
filters,
options,
hadoopConf,
- metrics,
- alluxioPathReplacementMap)
+ metrics)
}
override def supportFieldName(name: String): Boolean = {
diff --git a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
index ad0f9841b39..9103eb0f935 100644
--- a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
+++ b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -63,8 +63,7 @@ trait GpuDeltaParquetFileFormat extends GpuReadParquetFileFormat {
pushedFilters,
fileScan.rapidsConf,
fileScan.allMetrics,
- fileScan.queryUsesInputFile,
- fileScan.alluxioPathsMap)
+ fileScan.queryUsesInputFile)
}
override def buildReaderWithPartitionValuesAndMetrics(
@@ -75,8 +74,7 @@ trait GpuDeltaParquetFileFormat extends GpuReadParquetFileFormat {
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
super.buildReaderWithPartitionValuesAndMetrics(
sparkSession,
@@ -86,8 +84,7 @@ trait GpuDeltaParquetFileFormat extends GpuReadParquetFileFormat {
filters,
options,
hadoopConf,
- metrics,
- alluxioPathReplacementMap)
+ metrics)
}
override def supportFieldName(name: String): Boolean = {
diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala
index 77891864537..2ce38a2d6a1 100644
--- a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala
+++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -68,11 +68,8 @@ case class GpuDelta24xParquetFileFormat(
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
-
-
val dataReader = super.buildReaderWithPartitionValuesAndMetrics(
sparkSession,
dataSchema,
@@ -81,8 +78,7 @@ case class GpuDelta24xParquetFileFormat(
if (disablePushDown) Seq.empty else filters,
options,
hadoopConf,
- metrics,
- alluxioPathReplacementMap)
+ metrics)
val delVecs = broadcastDvMap
val maxDelVecScatterBatchSize = RapidsConf
diff --git a/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
index 088a2a788da..e32f1caafa0 100644
--- a/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
+++ b/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -65,8 +65,7 @@ case class GpuDeltaParquetFileFormat(
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
val dataReader = super.buildReaderWithPartitionValuesAndMetrics(
@@ -77,8 +76,7 @@ case class GpuDeltaParquetFileFormat(
filters,
options,
hadoopConf,
- metrics,
- alluxioPathReplacementMap)
+ metrics)
val delVecs = broadcastDvMap
val maxDelVecScatterBatchSize = RapidsConf
diff --git a/delta-lake/delta-spark350db143/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark350db143/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
index abaece5feb3..42831508c5b 100644
--- a/delta-lake/delta-spark350db143/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
+++ b/delta-lake/delta-spark350db143/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2024, NVIDIA CORPORATION.
+ * Copyright (c) 2024-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -65,8 +65,7 @@ case class GpuDeltaParquetFileFormat(
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
val dataReader = super.buildReaderWithPartitionValuesAndMetrics(
@@ -77,8 +76,7 @@ case class GpuDeltaParquetFileFormat(
filters,
options,
hadoopConf,
- metrics,
- alluxioPathReplacementMap)
+ metrics)
val delVecs = broadcastDvMap
val maxDelVecScatterBatchSize = RapidsConf
diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md
index 5519e56b419..ce703213ba5 100644
--- a/docs/additional-functionality/advanced_configs.md
+++ b/docs/additional-functionality/advanced_configs.md
@@ -19,16 +19,6 @@ For commonly used configurations and examples of setting options, please refer t
Name | Description | Default Value | Applicable at
-----|-------------|--------------|--------------
-spark.rapids.alluxio.automount.enabled|Enable the feature of auto mounting the cloud storage to Alluxio. It requires the Alluxio master is the same node of Spark driver node. The Alluxio master's host and port will be read from alluxio.master.hostname and alluxio.master.rpc.port(default: 19998) from ALLUXIO_HOME/conf/alluxio-site.properties, then replace a cloud path which matches spark.rapids.alluxio.bucket.regex like "s3://bar/b.csv" to "alluxio://0.1.2.3:19998/bar/b.csv", and the bucket "s3://bar" will be mounted to "/bar" in Alluxio automatically.|false|Runtime
-spark.rapids.alluxio.bucket.regex|A regex to decide which bucket should be auto-mounted to Alluxio. E.g. when setting as "^s3://bucket.*", the bucket which starts with "s3://bucket" will be mounted to Alluxio and the path "s3://bucket-foo/a.csv" will be replaced to "alluxio://0.1.2.3:19998/bucket-foo/a.csv". It's only valid when setting spark.rapids.alluxio.automount.enabled=true. The default value matches all the buckets in "s3://" or "s3a://" scheme.|^s3a{0,1}://.*|Runtime
-spark.rapids.alluxio.home|The Alluxio installation home path or link to the installation home path. |/opt/alluxio|Startup
-spark.rapids.alluxio.large.file.threshold|The threshold is used to identify whether average size of files is large when reading from S3. If reading large files from S3 and the disks used by Alluxio are slow, directly reading from S3 is better than reading caches from Alluxio, because S3 network bandwidth is faster than local disk. This improvement takes effect when spark.rapids.alluxio.slow.disk is enabled.|67108864|Runtime
-spark.rapids.alluxio.master|The Alluxio master hostname. If not set, read Alluxio master URL from spark.rapids.alluxio.home locally. This config is useful when Alluxio master and Spark driver are not co-located.||Startup
-spark.rapids.alluxio.master.port|The Alluxio master port. If not set, read Alluxio master port from spark.rapids.alluxio.home locally. This config is useful when Alluxio master and Spark driver are not co-located.|19998|Startup
-spark.rapids.alluxio.pathsToReplace|List of paths to be replaced with corresponding Alluxio scheme. E.g. when configure is set to "s3://foo->alluxio://0.1.2.3:19998/foo,gs://bar->alluxio://0.1.2.3:19998/bar", it means: "s3://foo/a.csv" will be replaced to "alluxio://0.1.2.3:19998/foo/a.csv" and "gs://bar/b.csv" will be replaced to "alluxio://0.1.2.3:19998/bar/b.csv". To use this config, you have to mount the buckets to Alluxio by yourself. If you set this config, spark.rapids.alluxio.automount.enabled won't be valid.|None|Startup
-spark.rapids.alluxio.replacement.algo|The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME and TASK_TIME are the valid options. CONVERT_TIME indicates that we do it when we convert it to a GPU file read, this has extra overhead of creating an entirely new file index, which requires listing the files and getting all new file info from Alluxio. TASK_TIME replaces the path as late as possible inside of the task. By waiting and replacing it at task time, it just replaces the path without fetching the file information again, this is faster but doesn't update locality information if that has a bit impact on performance.|TASK_TIME|Runtime
-spark.rapids.alluxio.slow.disk|Indicates whether the disks used by Alluxio are slow. If it's true and reading S3 large files, Rapids Accelerator reads from S3 directly instead of reading from Alluxio caches. Refer to spark.rapids.alluxio.large.file.threshold which defines a threshold that identifying whether files are large. Typically, it's slow disks if speed is less than 300M/second. If using convert time spark.rapids.alluxio.replacement.algo, this may not apply to all file types like Delta files|true|Runtime
-spark.rapids.alluxio.user|Alluxio user is set on the Alluxio client, which is used to mount or get information. By default it should be the user that running the Alluxio processes. The default value is ubuntu.|ubuntu|Runtime
spark.rapids.filecache.allowPathRegexp|A regular expression to decide which paths will be cached when the file cache is enabled. If this is not set, then all paths are allowed to cache. If a path is allowed by this regexp but blocked by spark.rapids.filecache.blockPathRegexp, then the path is blocked to cache.|None|Startup
spark.rapids.filecache.blockPathRegexp|A regular expression to decide which paths will not be cached when the file cache is enabled. If a path is blocked by this regexp but is allowed by spark.rapids.filecache.allowPathRegexp, then the path is blocked.|None|Startup
spark.rapids.filecache.checkStale|Controls whether the cached is checked for being out of date with respect to the input file. When enabled, the data that has been cached locally for a file will be invalidated if the file is updated after being cached. This feature is only necessary if an input file for a Spark application can be changed during the lifetime of the application. If an individual input file will not be overwritten during the Spark application then performance may be improved by setting this to false.|true|Startup
@@ -69,7 +59,7 @@ Name | Description | Default Value | Applicable at
spark.rapids.sql.castFloatToString.enabled|Casting from floating point types to string on the GPU returns results that have a different precision than the default results of Spark.|true|Runtime
spark.rapids.sql.castStringToFloat.enabled|When set to true, enables casting from strings to float types (float, double) on the GPU. Currently hex values aren't supported on the GPU. Also note that casting from string to float types on the GPU returns incorrect results when the string represents any number "1.7976931348623158E308" <= x < "1.7976931348623159E308" and "-1.7976931348623158E308" >= x > "-1.7976931348623159E308" in both these cases the GPU returns Double.MaxValue while CPU returns "+Infinity" and "-Infinity" respectively|true|Runtime
spark.rapids.sql.castStringToTimestamp.enabled|When set to true, casting from string to timestamp is supported on the GPU. The GPU only supports a subset of formats when casting strings to timestamps. Refer to the CAST documentation for more details.|false|Runtime
-spark.rapids.sql.coalescing.reader.numFilterParallel|This controls the number of files the coalescing reader will run in each thread when it filters blocks for reading. If this value is greater than zero the files will be filtered in a multithreaded manner where each thread filters the number of files set by this config. If this is set to zero the files are filtered serially. This uses the same thread pool as the multithreaded reader, see spark.rapids.sql.multiThreadedRead.numThreads. Note that filtering multithreaded is useful with Alluxio.|0|Runtime
+spark.rapids.sql.coalescing.reader.numFilterParallel|This controls the number of files the coalescing reader will run in each thread when it filters blocks for reading. If this value is greater than zero the files will be filtered in a multithreaded manner where each thread filters the number of files set by this config. If this is set to zero the files are filtered serially. This uses the same thread pool as the multithreaded reader, see spark.rapids.sql.multiThreadedRead.numThreads.|0|Runtime
spark.rapids.sql.concurrentWriterPartitionFlushSize|The flush size of the concurrent writer cache in bytes for each partition. If specified spark.sql.maxConcurrentOutputFileWriters, use concurrent writer to write data. Concurrent writer first caches data for each partition and begins to flush the data if it finds one partition with a size that is greater than or equal to this config. The default value is 0, which will try to select a size based off of file type specific configs. E.g.: It uses `write.parquet.row-group-size-bytes` config for Parquet type and `orc.stripe.size` config for Orc type. If the value is greater than 0, will use this positive value.Max value may get better performance but not always, because concurrent writer uses spillable cache and big value may cause more IO swaps.|0|Runtime
spark.rapids.sql.csv.read.decimal.enabled|CSV reading is not 100% compatible when reading decimals.|false|Runtime
spark.rapids.sql.csv.read.double.enabled|CSV reading is not 100% compatible when reading doubles.|true|Runtime
diff --git a/pom.xml b/pom.xml
index dba06a7d08c..401ddb601b5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1,6 +1,6 @@
- org.alluxio
- alluxio-shaded-client
- ${alluxio.client.version}
- provided
-
- org.alluxio
- alluxio-shaded-client
- ${alluxio.client.version}
- provided
-
- org.alluxio
- alluxio-shaded-client
-
org.mockito
mockito-core
diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml
index 2b0a62a5b90..a4225585439 100644
--- a/sql-plugin/pom.xml
+++ b/sql-plugin/pom.xml
@@ -1,6 +1,6 @@
- org.alluxio
- alluxio-shaded-client
-
org.mockito
mockito-core
diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java
index b32e5e755cb..a928f702bb0 100644
--- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java
+++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2022-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -358,8 +358,6 @@ protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles,
false, // ignoreMissingFiles
false, // ignoreCorruptFiles
false, // useFieldId
- scala.collection.immutable.Map$.MODULE$.empty(), // alluxioPathReplacementMap
- false, // alluxioReplacementTaskTime
queryUsesInputFile,
true, // keepReadsInOrder
new CombineConf(
diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java
index b7544675d41..cd5694370b4 100644
--- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java
+++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2022-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -270,7 +270,7 @@ private scala.Tuple3 multiFileReadCheck(ReadTask r
.toArray(String[]::new);
// Get the final decision for the subtype of the Rapids reader.
boolean useMultiThread = MultiFileReaderUtils.useMultiThreadReader(
- canUseCoalescing, canUseMultiThread, files, allCloudSchemes, false);
+ canUseCoalescing, canUseMultiThread, files, allCloudSchemes);
return scala.Tuple3.apply(canAccelerateRead, useMultiThread, ff);
}
}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioCfgUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioCfgUtils.scala
deleted file mode 100644
index 7cdab3e58b6..00000000000
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioCfgUtils.scala
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright (c) 2022, 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.
- */
-
-package com.nvidia.spark.rapids
-
-import org.apache.spark.sql.execution.datasources.FileFormat
-import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
-
-/*
- * Utils for the alluxio configurations.
- * If alluxio feature is disabled, we can tolerate the client jar is absent.
- * Use the following pattern to avoid `NoClassDefFoundError` if alluxio feature is disabled:
- * if(AlluxioCfgUtils.enabledAlluxio* functions) {
- * // AlluxioCfgUtils does not import any alluxio class
- * // Only AlluxioUtils imports alluxio classes
- * AlluxioUtils.doSomething;
- * }
- */
-object AlluxioCfgUtils {
- def checkAlluxioNotSupported(rapidsConf: RapidsConf): Unit = {
- if (rapidsConf.isParquetPerFileReadEnabled &&
- (rapidsConf.getAlluxioAutoMountEnabled || rapidsConf.getAlluxioPathsToReplace.isDefined)) {
- throw new IllegalArgumentException("Alluxio is currently not supported with the PERFILE " +
- "reader, please use one of the other reader types.")
- }
- }
-
- /**
- * Returns whether alluxio convert time algorithm should be enabled
- * Note: should also check whether the auto-mount or replace path is enabled.
- *
- * @param conf the rapids conf
- * @return Returns whether alluxio convert time algorithm should be enabled
- */
- def enabledAlluxioReplacementAlgoConvertTime(conf: RapidsConf): Boolean = {
- conf.isAlluxioReplacementAlgoConvertTime &&
- (conf.getAlluxioAutoMountEnabled || conf.getAlluxioPathsToReplace.isDefined)
- }
-
- def enabledAlluxioReplacementAlgoTaskTime(conf: RapidsConf): Boolean = {
- conf.isAlluxioReplacementAlgoTaskTime &&
- (conf.getAlluxioAutoMountEnabled || conf.getAlluxioPathsToReplace.isDefined)
- }
-
- def isAlluxioAutoMountTaskTime(rapidsConf: RapidsConf,
- fileFormat: FileFormat): Boolean = {
- rapidsConf.getAlluxioAutoMountEnabled && rapidsConf.isAlluxioReplacementAlgoTaskTime &&
- fileFormat.isInstanceOf[ParquetFileFormat]
- }
-
- def isAlluxioPathsToReplaceTaskTime(rapidsConf: RapidsConf,
- fileFormat: FileFormat): Boolean = {
- rapidsConf.getAlluxioPathsToReplace.isDefined && rapidsConf.isAlluxioReplacementAlgoTaskTime &&
- fileFormat.isInstanceOf[ParquetFileFormat]
- }
-
- def isConfiguredReplacementMap(conf: RapidsConf): Boolean = {
- conf.getAlluxioPathsToReplace.isDefined
- }
-}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioConfigReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioConfigReader.scala
deleted file mode 100644
index 50a681f6218..00000000000
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioConfigReader.scala
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright (c) 2022-2023, 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.
- */
-package com.nvidia.spark.rapids
-
-import java.io.FileNotFoundException
-import java.util.Properties
-
-import scala.io.BufferedSource
-
-/**
- * Alluxio master address and port reader.
- * It reads from `/opt/alluxio/conf/alluxio-site.properties`
- */
-class AlluxioConfigReader {
-
- def readAlluxioMasterAndPort(conf: RapidsConf): (String, String) = {
- if (conf.getAlluxioMaster.isEmpty) {
- // Read from local Alluxio home
- readMasterAndPort(conf.getAlluxioHome)
- } else {
- // Alluxio master separately deployed and not co-located with Spark Driver.
- // Like: EKS Env,
- (conf.getAlluxioMaster, conf.getAlluxioMasterPort.toString)
- }
- }
-
- // By default, read from /opt/alluxio, refer to `spark.rapids.alluxio.home` config in `RapidsConf`
- // The default port is 19998
- private[rapids] def readMasterAndPort(homePath: String): (String, String) = {
- var buffered_source: BufferedSource = null
- try {
- buffered_source = scala.io.Source.fromFile(homePath + "/conf/alluxio-site.properties")
- val prop: Properties = new Properties()
- prop.load(buffered_source.bufferedReader())
- val alluxio_master = prop.getProperty("alluxio.master.hostname")
- if (alluxio_master == null) {
- throw new RuntimeException(
- s"Can't find alluxio.master.hostname from $homePath/conf/alluxio-site.properties.")
- }
-
- val alluxio_port = prop.getProperty("alluxio.master.rpc.port", "19998")
- (alluxio_master, alluxio_port)
- } catch {
- case _: FileNotFoundException =>
- throw new RuntimeException(s"Alluxio config file not found in " +
- s"$homePath/conf/alluxio-site.properties, " +
- "the default value of `spark.rapids.alluxio.home` is /opt/alluxio, " +
- "please create a link `/opt/alluxio` to Alluxio installation home, " +
- "or set `spark.rapids.alluxio.home` to Alluxio installation home")
- } finally {
- if (buffered_source != null) buffered_source.close
- }
- }
-}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioFS.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioFS.scala
deleted file mode 100644
index 88994bae182..00000000000
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioFS.scala
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright (c) 2022-2023, 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.
- */
-package com.nvidia.spark.rapids
-
-import scala.collection.JavaConverters.mapAsScalaMapConverter
-import scala.collection.mutable
-
-import alluxio.AlluxioURI
-import alluxio.conf.{AlluxioProperties, InstancedConfiguration, PropertyKey}
-import alluxio.grpc.MountPOptions
-import com.nvidia.spark.rapids.Arm.withResource
-
-/**
- * interfaces for Alluxio file system.
- * Currently contains interfaces:
- * get mount points
- * mount
- */
-class AlluxioFS {
- private var masterHost: String = _
- private var masterPort: Int = _
- private var masterHostAndPort: Option[String] = None
-
- private var alluxioUser: String = ""
- private var s3AccessKey: Option[String] = None
- private var s3SecretKey: Option[String] = None
-
- def setHostAndPort(masterHost: String, masterPort: Int): Unit = {
- this.masterHost = masterHost
- this.masterPort = masterPort
- this.masterHostAndPort = Some(masterHost + ":" + masterPort)
- }
-
- def setUserAndKeys(alluxioUser: String, s3AccessKey: Option[String],
- s3SecretKey: Option[String]): Unit = {
- this.alluxioUser = alluxioUser
- this.s3AccessKey = s3AccessKey
- this.s3SecretKey = s3SecretKey
- }
-
- private def getS3ClientConf(): InstancedConfiguration = {
- val p = new AlluxioProperties()
- p.set(PropertyKey.MASTER_HOSTNAME, masterHost)
- p.set(PropertyKey.MASTER_RPC_PORT, masterPort)
- s3AccessKey.foreach(access => p.set(PropertyKey.S3A_ACCESS_KEY, access))
- s3SecretKey.foreach(secret => p.set(PropertyKey.S3A_SECRET_KEY, secret))
- p.set(PropertyKey.SECURITY_LOGIN_USERNAME, alluxioUser)
- new InstancedConfiguration(p)
- }
-
- /**
- * Get S3 mount points by Alluxio client
- *
- * @return mount points map, key of map is Alluxio path, value of map is S3 path.
- * E.g.: returns a map: {'/bucket_1': 's3://bucket_1'}
- */
- def getExistingMountPoints(): mutable.Map[String, String] = {
- val conf = getS3ClientConf()
- // get s3 mount points by Alluxio client
- withResource(alluxio.client.file.FileSystem.Factory.create(conf)) { fs =>
- val mountTable = fs.getMountTable
- mountTable.asScala.filter { case (_, mountPoint) =>
- // checked the alluxio code, the type should be s3
- // anyway let's keep both of them
- mountPoint.getUfsType == "s3" || mountPoint.getUfsType == "s3a"
- }.map { case (alluxioPath, s3Point) =>
- (alluxioPath, s3Point.getUfsUri)
- }
- }
- }
-
- /**
- * Mount an S3 path to Alluxio
- *
- * @param alluxioPath Alluxio path
- * @param s3Path S3 path
- */
- def mount(alluxioPath: String, s3Path: String): Unit = {
- val conf = getS3ClientConf()
- withResource(alluxio.client.file.FileSystem.Factory.create(conf)) { fs =>
- val mountOptionsBuilder = MountPOptions.newBuilder().setReadOnly(true)
- s3AccessKey.foreach(e => mountOptionsBuilder.putProperties("s3a.accessKeyId", e))
- s3SecretKey.foreach(e => mountOptionsBuilder.putProperties("s3a.secretKey", e))
- fs.mount(new AlluxioURI(alluxioPath), new AlluxioURI(s3Path),
- mountOptionsBuilder.build())
- }
- }
-
- def getMasterHostAndPort(): Option[String] = masterHostAndPort
-}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala
deleted file mode 100644
index b717eb994c3..00000000000
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala
+++ /dev/null
@@ -1,671 +0,0 @@
-/*
- * Copyright (c) 2022-2023, 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.
- */
-
-package com.nvidia.spark.rapids
-
-import java.io.FileNotFoundException
-
-import scala.util.control.NonFatal
-
-import com.nvidia.spark.rapids.shims.PartitionedFileUtilsShim
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.RuntimeConfig
-import org.apache.spark.sql.catalyst.expressions.{DynamicPruning, Expression, PlanExpression}
-import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileIndex, HadoopFsRelation, InMemoryFileIndex, PartitionDirectory, PartitionedFile, PartitionSpec}
-import org.apache.spark.sql.execution.datasources.rapids.GpuPartitioningUtils
-
-/*
- * Utilities for using Alluxio with the plugin for reading.
- * Currently we only support Alluxio with the Datasource v1 Parquet reader.
- * We currently support 2 different replacement algorithms:
- * CONVERT_TIME: this replaces the file path when we convert a FileSourceScanExec to
- * a GpuFileSourceScanExec. This will create an entirely new FileIndex and potentially
- * has to re-infer the partitioning if its not a FileIndex type we know. So this can
- * cause an extra list leaf files which for many files will run another job and thus
- * has additional overhead. This will update the file locations to be the
- * Alluxio specific ones if the data is already cached. In order to support the
- * input_file_name functionality we have to convert the alluxio:// path back to its
- * original url when we go to actually read the file.
- * TASK_TIME: this replaces the file path as late as possible on the task side when
- * we actually go to read the file. This makes is so that the original non-Alluxio
- * path gets reported for the input_file_name properly without having to convert
- * paths back to the original. This also has the benefit that it can be more performant
- * if it doesn't have to do the extra list leaf files, but you don't get the
- * locality information updated. So for small Alluxio clusters or with Spark
- * clusters short on task slots this may be a better fit.
- *
- * The way we do the actual replacement algorithm differs depending on the file reader
- * type we use: PERFILE, COALESCING or MULTITHREADED.
- * PERFILE is not supported with Alluxio due to not easily being able to fix up
- * input_file_name. We could but would require copying the FileScanRDD so skip for now.
- * The COALESCING reader is not support when input_file_name is requested so it falls
- * back to the MULTITHREADED reader if that is used, when input_file_name is not requested,
- * we replace the paths properly based on the replacement algorithm and don't have to worry
- * about calculating the original path. The MULTITHREADED reader supports input_file_name
- * so it handles calculating the original file path in the case of the convert time algorithm.
- * In order to do the replacement at task time and to output the original path for convert
- * time, we need to have a mapping of the original scheme to the alluxio scheme. This has been
- * made a parameter to many of the readers. With auto mount and task time replacement,
- * we make a pass through the files on the driver side in GpuFileSourceScanExec in order to
- * do the mounting before the tasks try to access alluxio.
- * Note that Delta Lake uses the input_file_name functionality to do things like
- * Updates and Deletes and will fail if the path has the alluxio:// in it.
- *
- * Below we support 2 configs to turn on Alluxio, we have the automount which uses a regex
- * to replace paths and then we have the config that specifies direct paths to replace and
- * user has to manually mount those.
- */
-object AlluxioUtils extends Logging {
- private val checkedAlluxioPath = scala.collection.mutable.HashSet[String]()
- private val ALLUXIO_SCHEME = "alluxio://"
- private val mountedBuckets: scala.collection.mutable.Map[String, String] =
- scala.collection.mutable.Map()
- private var alluxioPathsToReplaceMap: Option[Map[String, String]] = None
- private var alluxioBucketRegex: Option[String] = None
- private var isInitReplaceMap: Boolean = false
- private var isInitMountPointsForAutoMount: Boolean = false
- private var alluxioFS: AlluxioFS = new AlluxioFS()
- private var alluxioMasterAndPortReader = new AlluxioConfigReader()
-
- private def checkAlluxioMounted(
- hadoopConfiguration: Configuration,
- alluxio_path: String): Unit = {
- this.synchronized {
- if (!checkedAlluxioPath.contains(alluxio_path)) {
- val path = new Path(alluxio_path)
- val fs = path.getFileSystem(hadoopConfiguration)
- if (!fs.exists(path)) {
- throw new FileNotFoundException(
- s"Alluxio path $alluxio_path does not exist, maybe forgot to mount it")
- }
- logDebug(s"Alluxio path $alluxio_path is mounted")
- checkedAlluxioPath.add(alluxio_path)
- } else {
- logDebug(s"Alluxio path $alluxio_path already mounted")
- }
- }
- }
-
- // By default, read from /opt/alluxio, refer to `spark.rapids.alluxio.home` config in `RapidsConf`
- private def readAlluxioMasterAndPort(conf: RapidsConf): (String, String) = {
- alluxioMasterAndPortReader.readAlluxioMasterAndPort(conf)
- }
-
- // Read out alluxio.master.hostname, alluxio.master.rpc.port
- // from Alluxio's conf alluxio-site.properties
- // We require an environment variable "ALLUXIO_HOME"
- // This function will only read once from ALLUXIO/conf.
- private def initAlluxioInfo(conf: RapidsConf, hadoopConf: Configuration,
- runtimeConf: RuntimeConfig): Unit = {
- this.synchronized {
- // left outside isInit to allow changing at runtime
- AlluxioCfgUtils.checkAlluxioNotSupported(conf)
-
- if (AlluxioCfgUtils.isConfiguredReplacementMap(conf)) {
- // replace-map is enabled, if set this will invalid the auto-mount
- if (!isInitReplaceMap) {
- alluxioPathsToReplaceMap = getReplacementMapOption(conf)
- isInitReplaceMap = true
- }
- } else if (conf.getAlluxioAutoMountEnabled) {
- // auto-mount is enabled
- if (!isInitMountPointsForAutoMount) {
- val (alluxioMasterHostStr, alluxioMasterPortStr) = readAlluxioMasterAndPort(conf)
- alluxioBucketRegex = Some(conf.getAlluxioBucketRegex)
- // load mounted point by call Alluxio client.
- try {
- val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf)
- // get existing mount points
- alluxioFS.setHostAndPort(alluxioMasterHostStr, alluxioMasterPortStr.toInt)
- alluxioFS.setUserAndKeys(conf.getAlluxioUser, access_key, secret_key)
- val mountPoints = alluxioFS.getExistingMountPoints()
-
- mountPoints.foreach { case (alluxioPath, s3Path) =>
- // record map info from alluxio path to s3 path
- mountedBuckets(alluxioPath) = s3Path
- logInfo(s"Found mounted bucket $s3Path to $alluxioPath")
- }
- } catch {
- case NonFatal(e) => logWarning(s"Failed to get alluxio mount table", e)
- }
- isInitMountPointsForAutoMount = true
- }
- } else {
- // disabled Alluxio feature, do nothing
- }
- }
- }
-
- // The path should be like s3://bucket/... or s3a://bucket/...
- private def getSchemeAndBucketFromPath(path: String): (String, String) = {
- val i = path.split("://")
- val scheme = i(0)
- if (i.length <= 1) {
- throw new RuntimeException(s"path $path is not expected for Alluxio auto mount")
- }
- val bucket = i(1).split("/")(0)
- (scheme + "://", bucket)
- }
-
- // path is like "s3://foo/test...", it mounts bucket "foo" by calling the alluxio CLI
- // And we'll append --option to set access_key and secret_key if existing.
- // Suppose the key doesn't exist when using like Databricks's instance profile
- private def autoMountBucket(alluxioUser: String, scheme: String, bucket: String,
- s3AccessKey: Option[String], s3SecretKey: Option[String]): Unit = {
-
- // to match the output of alluxio fs mount, append / to remote_path
- // and add / before bucket name for absolute path in Alluxio
- val remote_path = scheme + bucket + "/"
- val local_bucket = "/" + bucket
- this.synchronized {
- if (!mountedBuckets.contains(local_bucket)) {
- try {
- // not mount yet, call mount command
- // we only support s3 or s3a bucket for now.
- // To support other cloud storage,
- // we need to support credential parameters for the others
- alluxioFS.setUserAndKeys(alluxioUser, s3AccessKey, s3SecretKey)
- alluxioFS.mount(local_bucket, remote_path)
- logInfo(s"Mounted bucket $remote_path to $local_bucket in Alluxio")
- mountedBuckets(local_bucket) = remote_path
- } catch {
- case NonFatal(e) =>
- throw new RuntimeException(s"Mount bucket $remote_path to $local_bucket failed", e)
- }
- } else if (mountedBuckets(local_bucket).equals(remote_path)) {
- logDebug(s"Already mounted bucket $remote_path to $local_bucket in Alluxio")
- } else {
- throw new RuntimeException(s"Found a same bucket name in $remote_path " +
- s"and ${mountedBuckets(local_bucket)}")
- }
- }
- }
-
- // first try to get fs.s3a.access.key from spark config
- // second try to get from environment variables
- private def getKeyAndSecret(
- hadoopConfiguration: Configuration,
- runtimeConf: RuntimeConfig): (Option[String], Option[String]) = {
- val hadoopAccessKey =
- hadoopConfiguration.get("fs.s3a.access.key")
- val hadoopSecretKey =
- hadoopConfiguration.get("fs.s3a.secret.key")
- if (hadoopAccessKey != null && hadoopSecretKey != null) {
- (Some(hadoopAccessKey), Some(hadoopSecretKey))
- } else {
- val accessKey = runtimeConf.getOption("spark.hadoop.fs.s3a.access.key")
- val secretKey = runtimeConf.getOption("spark.hadoop.fs.s3a.secret.key")
- if (accessKey.isDefined && secretKey.isDefined) {
- (accessKey, secretKey)
- } else {
- val envAccessKey = scala.util.Properties.envOrNone("AWS_ACCESS_KEY_ID")
- val envSecretKey = scala.util.Properties.envOrNone("AWS_ACCESS_SECRET_KEY")
- (envAccessKey, envSecretKey)
- }
- }
- }
-
- private def replaceSchemeWithAlluxio(file: String, scheme: String, masterPort: String): String = {
- // replace s3://foo/.. to alluxio://alluxioMasterHostAndPort/foo/...
- val newFile = file.replaceFirst(scheme, ALLUXIO_SCHEME + masterPort + "/")
- logDebug(s"Replace $file to $newFile")
- newFile
- }
-
- private def genFuncForPathReplacement(
- replaceMapOption: Option[Map[String, String]])
- : Option[Path => AlluxioPathReplaceConvertTime] = {
- if (replaceMapOption.isDefined) {
- Some((f: Path) => {
- val pathStr = f.toString
- val matchedSet = replaceMapOption.get.filter(a => pathStr.startsWith(a._1))
- if (matchedSet.size > 1) {
- // never reach here since replaceMap is a Map
- throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " +
- s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " +
- s"for each file path")
- } else if (matchedSet.size == 1) {
- val res = AlluxioPathReplaceConvertTime(
- new Path(pathStr.replaceFirst(matchedSet.head._1, matchedSet.head._2)),
- Some(matchedSet.head._1))
- logDebug(s"Specific path replacement, replacing paths with: $res")
- res
- } else {
- AlluxioPathReplaceConvertTime(f, None)
- }
- })
- } else {
- None
- }
- }
-
- private def genFuncForAutoMountReplacement(
- alluxioUser: String,
- runtimeConf: RuntimeConfig,
- hadoopConf: Configuration): Option[Path => AlluxioPathReplaceConvertTime] = {
- assert(alluxioFS.getMasterHostAndPort().isDefined)
- assert(alluxioBucketRegex.isDefined)
-
- Some((f: Path) => {
- val pathStr = f.toString
- val res = if (pathStr.matches(alluxioBucketRegex.get)) {
- val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf)
- val (scheme, bucket) = getSchemeAndBucketFromPath(pathStr)
- autoMountBucket(alluxioUser, scheme, bucket, access_key, secret_key)
- AlluxioPathReplaceConvertTime(
- new Path(replaceSchemeWithAlluxio(pathStr, scheme, alluxioFS.getMasterHostAndPort().get)),
- Some(scheme))
- } else {
- AlluxioPathReplaceConvertTime(f, None)
- }
- logDebug(s"Automount replacing paths: $res")
- res
- })
- }
-
- // Contains the file string to read and contains a boolean indicating if the
- // path was updated to an alluxio:// path.
- private case class AlluxioPathReplaceTaskTime(fileStr: String, wasReplaced: Boolean)
-
- // Contains the file Path to read and optionally contains the prefix of the original path.
- // The original path is needed when using the input_file_name option with the reader so
- // it reports the original path and not the alluxio version
- case class AlluxioPathReplaceConvertTime(filePath: Path, origPrefix: Option[String])
-
- // Replaces the file name with Alluxio one if it matches.
- // Returns a tuple with the file path and whether or not it replaced the
- // scheme with the Alluxio one.
- private def genFuncForTaskTimeReplacement(pathsToReplace: Map[String, String])
- : String => AlluxioPathReplaceTaskTime = {
- (pathStr: String) => {
- // pathsToReplace contain strings of exact paths to replace
- val matchedSet = pathsToReplace.filter { case (pattern, _) => pathStr.startsWith(pattern) }
- if (matchedSet.size > 1) {
- // never reach here since replaceMap is a Map
- throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " +
- s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " +
- s"for each file path")
- } else if (matchedSet.size == 1) {
- AlluxioPathReplaceTaskTime(
- pathStr.replaceFirst(matchedSet.head._1, matchedSet.head._2), wasReplaced = true)
- } else {
- AlluxioPathReplaceTaskTime(pathStr, wasReplaced = false)
- }
- }
- }
-
- private def getReplacementMapOption(conf: RapidsConf): Option[Map[String, String]] = {
- val alluxioPathsReplace: Option[Seq[String]] = conf.getAlluxioPathsToReplace
- // alluxioPathsReplace: Seq("key->value", "key1->value1")
- // turn the rules to the Map with eg
- // { s3://foo -> alluxio://0.1.2.3:19998/foo,
- // gs://bar -> alluxio://0.1.2.3:19998/bar }
- if (alluxioPathsReplace.isDefined) {
- alluxioPathsReplace.map(rules => {
- rules.map(rule => {
- val split = rule.split("->")
- if (split.size == 2) {
- split(0).trim -> split(1).trim
- } else {
- throw new IllegalArgumentException(s"Invalid setting for " +
- s"${RapidsConf.ALLUXIO_PATHS_REPLACE.key}")
- }
- }).toMap
- })
- } else {
- None
- }
- }
-
- private def getReplacementFunc(
- conf: RapidsConf,
- runtimeConf: RuntimeConfig,
- hadoopConf: Configuration): Option[Path => AlluxioPathReplaceConvertTime] = {
- if (conf.getAlluxioPathsToReplace.isDefined) {
- genFuncForPathReplacement(alluxioPathsToReplaceMap)
- } else if (conf.getAlluxioAutoMountEnabled) {
- genFuncForAutoMountReplacement(conf.getAlluxioUser, runtimeConf, hadoopConf)
- } else {
- None
- }
- }
-
- // assumes Alluxio directories already mounted at this point
- def updateFilesTaskTimeIfAlluxio(
- origFiles: Array[PartitionedFile],
- alluxioPathReplacementMap: Option[Map[String, String]])
- : Array[PartitionedFileInfoOptAlluxio] = {
- val res: Array[PartitionedFileInfoOptAlluxio] =
- alluxioPathReplacementMap.map { pathsToReplace =>
- replacePathInPartitionFileTaskTimeIfNeeded(pathsToReplace, origFiles)
- }.getOrElse(origFiles.map(PartitionedFileInfoOptAlluxio(_, None)))
- logDebug(s"Updated files at TASK_TIME for Alluxio: ${res.mkString(",")}")
- res
- }
-
- // Replaces the path if needed and returns the replaced path and optionally the
- // original file if it replaced the scheme with an Alluxio scheme.
- private def replacePathInPartitionFileTaskTimeIfNeeded(
- pathsToReplace: Map[String, String],
- files: Array[PartitionedFile]): Array[PartitionedFileInfoOptAlluxio] = {
- val replaceFunc = genFuncForTaskTimeReplacement(pathsToReplace)
-
- files.map { file =>
- val replacedFileInfo = replaceFunc(file.filePath.toString())
- if (replacedFileInfo.wasReplaced) {
- logDebug(s"TASK_TIME replaced ${file.filePath} with ${replacedFileInfo.fileStr}")
- PartitionedFileInfoOptAlluxio(
- PartitionedFileUtilsShim.newPartitionedFile(file.partitionValues,
- replacedFileInfo.fileStr, file.start, file.length),
- Some(file))
- } else {
- PartitionedFileInfoOptAlluxio(file, None)
- }
- }
- }
-
- // if auto-mount, use this to check if need replacements
- def autoMountIfNeeded(
- conf: RapidsConf,
- pds: Seq[PartitionDirectory],
- hadoopConf: Configuration,
- runtimeConf: RuntimeConfig): Option[Map[String, String]] = {
- val alluxioAutoMountEnabled = conf.getAlluxioAutoMountEnabled
- initAlluxioInfo(conf, hadoopConf, runtimeConf)
- if (alluxioAutoMountEnabled) {
- val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf)
- val replacedSchemes = pds.flatMap { pd =>
- pd.files.map(_.getPath.toString).flatMap { file =>
- if (file.matches(alluxioBucketRegex.get)) {
- val (scheme, bucket) = getSchemeAndBucketFromPath(file)
- autoMountBucket(conf.getAlluxioUser, scheme, bucket, access_key, secret_key)
- Some(scheme)
- } else {
- None
- }
- }
- }
- if (replacedSchemes.nonEmpty) {
- val alluxioMasterHostAndPort = alluxioFS.getMasterHostAndPort()
- Some(replacedSchemes.map(_ -> (ALLUXIO_SCHEME + alluxioMasterHostAndPort.get + "/")).toMap)
- } else {
- None
- }
- } else {
- None
- }
- }
-
- // If specified replace map, use this to check if need replacements
- def checkIfNeedsReplaced(
- conf: RapidsConf,
- pds: Seq[PartitionDirectory],
- hadoopConf: Configuration,
- runtimeConf: RuntimeConfig): Option[Map[String, String]] = {
- initAlluxioInfo(conf, hadoopConf, runtimeConf)
- val anyToReplace = pds.exists { pd =>
- pd.files.map(_.getPath.toString).exists { file =>
- val matchedSet = alluxioPathsToReplaceMap.get.filter(a => file.startsWith(a._1))
- if (matchedSet.size > 1) {
- // never reach here since replaceMap is a Map
- throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " +
- s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " +
- s"for each file path")
- } else if (matchedSet.size == 1) {
- true
- } else {
- false
- }
- }
- }
- if (anyToReplace) {
- alluxioPathsToReplaceMap
- } else {
- None
- }
- }
-
- // reverse the replacePathIfNeeded, returns a tuple of the file passed in and then if it
- // was replaced the original file
- def getOrigPathFromReplaced(pfs: Array[PartitionedFile],
- pathsToReplace: Map[String, String]): Array[PartitionedFileInfoOptAlluxio] = {
- pfs.map { pf =>
- val file = pf.filePath.toString()
- // pathsToReplace contain strings of exact paths to replace
- val matchedSet = pathsToReplace.filter { case (_, alluxioPattern) =>
- file.startsWith(alluxioPattern)
- }
- if (matchedSet.size > 1) {
- // never reach here since replaceMap is a Map
- throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " +
- s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " +
- s"for each file path")
- } else if (matchedSet.size == 1) {
- val replacedFile = file.replaceFirst(matchedSet.head._2, matchedSet.head._1)
- logDebug(s"getOrigPath replacedFile: $replacedFile")
- PartitionedFileInfoOptAlluxio(pf,
- Some(PartitionedFileUtilsShim.newPartitionedFile(pf.partitionValues, replacedFile,
- pf.start, file.length)))
- } else {
- PartitionedFileInfoOptAlluxio(pf, None)
- }
- }
- }
-
- // This is used when replacement algorithm is CONVERT_TIME and causes
- // a new lookup on the alluxio files. For unknown FileIndex types it can
- // also cause us to have to infer the partitioning again.
- def replacePathIfNeeded(
- conf: RapidsConf,
- relation: HadoopFsRelation,
- partitionFilters: Seq[Expression],
- dataFilters: Seq[Expression]): (FileIndex, Option[Map[String, String]]) = {
- val hadoopConf = relation.sparkSession.sparkContext.hadoopConfiguration
- val runtimeConf = relation.sparkSession.conf
- initAlluxioInfo(conf, hadoopConf, runtimeConf)
- val replaceFunc = getReplacementFunc(conf, runtimeConf, hadoopConf)
-
- val (location, allReplacedPrefixes) = if (replaceFunc.isDefined) {
- def replacePathsInPartitionSpec(spec: PartitionSpec): (PartitionSpec, Seq[String]) = {
- val partitionsWithPathsReplaced = spec.partitions.map { p =>
- val replacedPathAndPrefix = replaceFunc.get(p.path)
- (org.apache.spark.sql.execution.datasources.PartitionPath(p.values,
- replacedPathAndPrefix.filePath),
- replacedPathAndPrefix.origPrefix)
- }
- val paths = partitionsWithPathsReplaced.map(_._1)
- val replacedPrefixes = partitionsWithPathsReplaced.flatMap(_._2)
- (PartitionSpec(spec.partitionColumns, paths), replacedPrefixes)
- }
-
- def createNewFileIndexWithPathsReplaced(
- spec: PartitionSpec,
- rootPaths: Seq[Path]): (InMemoryFileIndex, Seq[String]) = {
- val (specAdjusted, replacedPrefixes) = replacePathsInPartitionSpec(spec)
- val replacedPathsAndIndicator = rootPaths.map(replaceFunc.get)
- val replacedPaths = replacedPathsAndIndicator.map(_.filePath)
- val didReplaceAnyRoots = replacedPathsAndIndicator.flatMap(_.origPrefix)
- val fi = new InMemoryFileIndex(
- relation.sparkSession,
- replacedPaths,
- relation.options,
- Option(relation.dataSchema),
- userSpecifiedPartitionSpec = Some(specAdjusted))
- (fi, didReplaceAnyRoots ++ replacedPrefixes)
- }
-
- // Before this change https://github.com/NVIDIA/spark-rapids/pull/6806,
- // if we know the type of file index, try to reuse as much of the existing
- // FileIndex as we can to keep from having to recalculate it and potentially
- // mess it up. But this causes always reading old files, see the issue of this PR.
- //
- // Now, because we have the Task time replacement algorithm,
- // just fall back to inferring partitioning for all FileIndex types except
- // CatalogFileIndex.
- // We use this approach to handle all the file index types for all CSPs like Databricks.
- relation.location match {
- case cfi: CatalogFileIndex =>
- logDebug("Handling CatalogFileIndex")
- val memFI = cfi.filterPartitions(Nil)
- createNewFileIndexWithPathsReplaced(memFI.partitionSpec(), memFI.rootPaths)
- case _ =>
- logDebug(s"Handling file index type: ${relation.location.getClass}")
-
- // With the base Spark FileIndex type we don't know how to modify it to
- // just replace the paths so we have to try to recompute.
- def isDynamicPruningFilter(e: Expression): Boolean = {
- e.isInstanceOf[DynamicPruning] || e.find(_.isInstanceOf[PlanExpression[_]]).isDefined
- }
-
- val partitionDirs = relation.location.listFiles(
- partitionFilters.filterNot(isDynamicPruningFilter), dataFilters)
-
- // replace all of input files
- val inputFilesAndDidReplace = partitionDirs.flatMap(partitionDir => {
- partitionDir.files.map(f => replaceFunc.get(f.getPath))
- })
- val inputFiles = inputFilesAndDidReplace.map(_.filePath)
- val didReplaceAny = inputFilesAndDidReplace.flatMap(_.origPrefix)
-
- // replace all of rootPaths which are already unique
- val rootPathsAndDidReplace = relation.location.rootPaths.map(replaceFunc.get)
- val rootPaths = rootPathsAndDidReplace.map(_.filePath)
- val rootPathsDidReplace = rootPathsAndDidReplace.flatMap(_.origPrefix)
-
- // check the alluxio paths in root paths exist or not
- // throw out an exception to stop the job when any of them is not mounted
- if (alluxioPathsToReplaceMap.isDefined) {
- rootPaths.foreach { rootPath =>
- alluxioPathsToReplaceMap.get.values.
- find(value => rootPath.toString.startsWith(value)).
- foreach(matched => checkAlluxioMounted(hadoopConf, matched))
- }
- }
-
- val parameters: Map[String, String] = relation.options
-
- // infer PartitionSpec
- val (partitionSpec, replacedBasePath) = GpuPartitioningUtils.inferPartitioning(
- relation.sparkSession,
- rootPaths,
- inputFiles,
- parameters,
- Option(relation.dataSchema),
- replaceFunc.get)
-
- val allReplacedPrefixes = didReplaceAny ++ rootPathsDidReplace ++ replacedBasePath
- // generate a new InMemoryFileIndex holding paths with alluxio schema
- val fi = new InMemoryFileIndex(
- relation.sparkSession,
- inputFiles,
- parameters,
- Option(relation.dataSchema),
- userSpecifiedPartitionSpec = Some(partitionSpec))
- (fi, allReplacedPrefixes)
- }
- } else {
- (relation.location, Seq.empty)
- }
- val mapIfReplacedPaths = if (allReplacedPrefixes.nonEmpty) {
- // with alluxio.automount.enabled we only have a regex so we need to track
- // the exact schemes we replaced in order to set the input_file_name properly,
- // with the alluxio.pathsToReplace it already contains the exact paths
- if (conf.getAlluxioAutoMountEnabled) {
- Some(allReplacedPrefixes.map(
- _ -> (ALLUXIO_SCHEME + alluxioFS.getMasterHostAndPort().get + "/")).toMap)
- } else {
- alluxioPathsToReplaceMap
- }
- } else {
- None
- }
- (location, mapIfReplacedPaths)
- }
-
- // If reading large s3 files on a cluster with slower disks, skip using Alluxio.
- def shouldReadDirectlyFromS3(rapidsConf: RapidsConf, pds: Seq[PartitionDirectory]): Boolean = {
- if (!rapidsConf.enableAlluxioSlowDisk) {
- logInfo(s"Skip reading directly from S3 because spark.rapids.alluxio.slow.disk is disabled")
- false
- } else {
- val filesWithoutDir = pds.flatMap(pd => pd.files).filter { file =>
- // skip directory
- !file.isDirectory
- }
-
- val files = filesWithoutDir.filter { f =>
- /**
- * Determines whether a file should be calculated for the average file size.
- * This is used to filter out some unrelated files,
- * such as transaction log files in the Delta file type.
- * However Delta files has other unrelated
- * files such as old regular parquet files.
- * Limitation: This is not OK for Delta file type, json file type, Avro file type......
- * Currently only care about parquet and orc files.
- * Note: It's hard to extract this into a method, because in Databricks 312 the files in
- * `PartitionDirectory` are in type of
- * `org.apache.spark.sql.execution.datasources.SerializableFileStatus`
- * instead of `org.apache.hadoop.fs.FileStatus`
- */
- f.getPath.getName.endsWith(".parquet") || f.getPath.getName.endsWith(".orc")
- }
-
- val totalSize = files.map(_.getLen).sum
-
- val avgSize = if (files.isEmpty) 0 else totalSize / files.length
- if (avgSize > rapidsConf.getAlluxioLargeFileThreshold) {
- // if files are large
- logInfo(s"Reading directly from S3, " +
- s"average file size $avgSize > threshold ${rapidsConf.getAlluxioLargeFileThreshold}")
- true
- } else {
- logInfo(s"Skip reading directly from S3, " +
- s"average file size $avgSize <= threshold ${rapidsConf.getAlluxioLargeFileThreshold}")
- false
- }
- }
- }
-
-
- /**
- * For test purpose only
- */
- def setAlluxioFS(alluxioFSMock: AlluxioFS): Unit = {
- alluxioFS = alluxioFSMock
- }
-
- /**
- * For test purpose only
- */
- def setAlluxioMasterAndPortReader(
- alluxioMasterAndPortReaderMock: AlluxioConfigReader): Unit = {
- alluxioMasterAndPortReader = alluxioMasterAndPortReaderMock
- }
-
- /**
- * For test purpose only
- */
- def resetInitInfo(): Unit = {
- isInitReplaceMap = false
- isInitMountPointsForAutoMount = false
- alluxioPathsToReplaceMap = None
- alluxioBucketRegex = None
- mountedBuckets.clear()
- }
-}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala
index ab02d1f0eea..6d8ab11c60c 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2021-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2021-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -69,8 +69,6 @@ object SingleHMBAndMeta {
trait HostMemoryBuffersWithMetaDataBase {
// PartitionedFile to be read
def partitionedFile: PartitionedFile
- // Original PartitionedFile if path was replaced with Alluxio
- def origPartitionedFile: Option[PartitionedFile] = None
// An array of BlockChunk(HostMemoryBuffer and its data size) read from PartitionedFile
def memBuffersAndSizes: Array[SingleHMBAndMeta]
// Total bytes read
@@ -174,16 +172,12 @@ object MultiFileReaderUtils {
filePaths.exists(fp => cloudSchemes.contains(fp.getScheme))
}
- // If Alluxio is enabled and we do task time replacement we have to take that
- // into account here so we use the Coalescing reader instead of the MultiThreaded reader.
def useMultiThreadReader(
coalescingEnabled: Boolean,
multiThreadEnabled: Boolean,
files: Array[String],
- cloudSchemes: Set[String],
- anyAlluxioPathsReplaced: Boolean = false): Boolean =
- !coalescingEnabled || (multiThreadEnabled &&
- (!anyAlluxioPathsReplaced && hasPathInCloud(files, cloudSchemes)))
+ cloudSchemes: Set[String]): Boolean =
+ !coalescingEnabled || (multiThreadEnabled && hasPathInCloud(files, cloudSchemes))
}
/**
@@ -193,14 +187,11 @@ object MultiFileReaderUtils {
* @param sqlConf the SQLConf
* @param broadcastedConf the Hadoop configuration
* @param rapidsConf the Rapids configuration
- * @param alluxioPathReplacementMap Optional map containing mapping of DFS
- * scheme to Alluxio scheme
*/
abstract class MultiFilePartitionReaderFactoryBase(
@transient sqlConf: SQLConf,
broadcastedConf: Broadcast[SerializableConfiguration],
- @transient rapidsConf: RapidsConf,
- alluxioPathReplacementMap: Option[Map[String, String]] = None)
+ @transient rapidsConf: RapidsConf)
extends PartitionReaderFactory with Logging {
protected val maxReadBatchSizeRows: Int = rapidsConf.maxReadBatchSizeRows
@@ -283,7 +274,7 @@ abstract class MultiFilePartitionReaderFactoryBase(
/** for testing */
private[rapids] def useMultiThread(filePaths: Array[String]): Boolean =
MultiFileReaderUtils.useMultiThreadReader(canUseCoalesceFilesReader,
- canUseMultiThreadReader, filePaths, allCloudSchemes, alluxioPathReplacementMap.isDefined)
+ canUseMultiThreadReader, filePaths, allCloudSchemes)
}
/**
@@ -310,11 +301,6 @@ abstract class FilePartitionReaderBase(conf: Configuration, execMetrics: Map[Str
}
}
-// Contains the actual file path to read from and then an optional original path if its read from
-// Alluxio. To make it transparent to the user, we return the original non-Alluxio path
-// for input_file_name.
-case class PartitionedFileInfoOptAlluxio(toRead: PartitionedFile, original: Option[PartitionedFile])
-
case class CombineConf(
combineThresholdSize: Long, // The size to combine to when combining small files
combineWaitTime: Int) // The amount of time to wait for other files ready for combination.
@@ -334,8 +320,6 @@ case class CombineConf(
* @param filters push down filters
* @param execMetrics the metrics
* @param ignoreCorruptFiles Whether to ignore corrupt files when GPU failed to decode the files
- * @param alluxioPathReplacementMap Map containing mapping of DFS scheme to Alluxio scheme
- * @param alluxioReplacementTaskTime Whether the Alluxio replacement algorithm is set to task time
* @param keepReadsInOrder Whether to require the files to be read in the same order as Spark.
* Defaults to true for formats that don't explicitly handle this.
* @param combineConf configs relevant to combination
@@ -350,8 +334,6 @@ abstract class MultiFileCloudPartitionReaderBase(
maxReadBatchSizeRows: Int,
maxReadBatchSizeBytes: Long,
ignoreCorruptFiles: Boolean = false,
- alluxioPathReplacementMap: Map[String, String] = Map.empty,
- alluxioReplacementTaskTime: Boolean = false,
keepReadsInOrder: Boolean = true,
combineConf: CombineConf = CombineConf(-1, -1))
extends FilePartitionReaderBase(conf, execMetrics) {
@@ -369,23 +351,9 @@ abstract class MultiFileCloudPartitionReaderBase(
// like in the case of a limit call and we don't read all files
private var fcs: ExecutorCompletionService[HostMemoryBuffersWithMetaDataBase] = null
- private val files: Array[PartitionedFileInfoOptAlluxio] = {
- if (alluxioPathReplacementMap.nonEmpty) {
- if (alluxioReplacementTaskTime) {
- AlluxioUtils.updateFilesTaskTimeIfAlluxio(inputFiles, Some(alluxioPathReplacementMap))
- } else {
- // was done at CONVERT_TIME, need to recalculate the original path to set for
- // input_file_name
- AlluxioUtils.getOrigPathFromReplaced(inputFiles, alluxioPathReplacementMap)
- }
- } else {
- inputFiles.map(PartitionedFileInfoOptAlluxio(_, None))
- }
- }
-
private def initAndStartReaders(): Unit = {
// limit the number we submit at once according to the config if set
- val limit = math.min(maxNumFileProcessed, files.length)
+ val limit = math.min(maxNumFileProcessed, inputFiles.length)
val tc = TaskContext.get
if (!keepReadsInOrder) {
logDebug("Not keeping reads in order")
@@ -404,25 +372,25 @@ abstract class MultiFileCloudPartitionReaderBase(
// the files in the future. ie try to start some of the larger files but we may not want
// them all to be large
for (i <- 0 until limit) {
- val file = files(i)
- logDebug(s"MultiFile reader using file ${file.toRead}, orig file is ${file.original}")
+ val file = inputFiles(i)
+ logDebug(s"MultiFile reader using file $file")
if (!keepReadsInOrder) {
- val futureRunner = fcs.submit(getBatchRunner(tc, file.toRead, file.original, conf, filters))
+ val futureRunner = fcs.submit(getBatchRunner(tc, file, conf, filters))
tasks.add(futureRunner)
} else {
// Add these in the order as we got them so that we can make sure
// we process them in the same order as CPU would.
val threadPool = MultiFileReaderThreadPool.getOrCreateThreadPool(numThreads)
- tasks.add(threadPool.submit(getBatchRunner(tc, file.toRead, file.original, conf, filters)))
+ tasks.add(threadPool.submit(getBatchRunner(tc, file, conf, filters)))
}
}
// queue up any left to add once others finish
- for (i <- limit until files.length) {
- val file = files(i)
- tasksToRun.enqueue(getBatchRunner(tc, file.toRead, file.original, conf, filters))
+ for (i <- limit until inputFiles.length) {
+ val file = inputFiles(i)
+ tasksToRun.enqueue(getBatchRunner(tc, file, conf, filters))
}
isInitted = true
- filesToRead = files.length
+ filesToRead = inputFiles.length
}
// Each format should implement combineHMBs and canUseCombine if they support combining
@@ -441,7 +409,6 @@ abstract class MultiFileCloudPartitionReaderBase(
*
* @param tc task context to use
* @param file file to be read
- * @param origFile optional original unmodified file if replaced with Alluxio
* @param conf the Configuration parameters
* @param filters push down filters
* @return Callable[HostMemoryBuffersWithMetaDataBase]
@@ -449,7 +416,6 @@ abstract class MultiFileCloudPartitionReaderBase(
def getBatchRunner(
tc: TaskContext,
file: PartitionedFile,
- origFile: Option[PartitionedFile],
conf: Configuration,
filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase]
@@ -618,8 +584,7 @@ abstract class MultiFileCloudPartitionReaderBase(
TrampolineUtil.incBytesRead(inputMetrics, fileBufsAndMeta.bytesRead)
// this is combine mode so input file shouldn't be used at all but update to
// what would be closest so we at least don't have same file as last batch
- val inputFileToSet =
- fileBufsAndMeta.origPartitionedFile.getOrElse(fileBufsAndMeta.partitionedFile)
+ val inputFileToSet = fileBufsAndMeta.partitionedFile
InputFileUtils.setInputFileBlock(
inputFileToSet.filePath.toString(),
inputFileToSet.start,
@@ -663,12 +628,7 @@ abstract class MultiFileCloudPartitionReaderBase(
}
TrampolineUtil.incBytesRead(inputMetrics, fileBufsAndMeta.bytesRead)
- // if we replaced the path with Alluxio, set it to the original filesystem file
- // since Alluxio replacement is supposed to be transparent to the user
- // Note that combine mode would have fallen back to not use combine mode if
- // the inputFile was required.
- val inputFileToSet =
- fileBufsAndMeta.origPartitionedFile.getOrElse(fileBufsAndMeta.partitionedFile)
+ val inputFileToSet = fileBufsAndMeta.partitionedFile
InputFileUtils.setInputFileBlock(
inputFileToSet.filePath.toString(),
inputFileToSet.start,
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala
index ababb052f27..550f6bc5591 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -2150,7 +2150,6 @@ class MultiFileCloudOrcPartitionReader(
override def getBatchRunner(
tc: TaskContext,
file: PartitionedFile,
- origFile: Option[PartitionedFile],
conf: Configuration,
filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = {
new ReadBatchRunner(tc, file, conf, filters)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala
index 03eb48de6fb..f37c00b8f5a 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -122,11 +122,11 @@ case class GpuParquetScan(
logInfo("Using the original per file parquet reader")
GpuParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf,
dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics,
- options.asScala.toMap, None)
+ options.asScala.toMap)
} else {
GpuParquetMultiFilePartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf,
dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics,
- queryUsesInputFile, None)
+ queryUsesInputFile)
}
}
@@ -1066,10 +1066,8 @@ case class GpuParquetMultiFilePartitionReaderFactory(
filters: Array[Filter],
@transient rapidsConf: RapidsConf,
metrics: Map[String, GpuMetric],
- queryUsesInputFile: Boolean,
- alluxioPathReplacementMap: Option[Map[String, String]])
- extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf,
- rapidsConf, alluxioPathReplacementMap) {
+ queryUsesInputFile: Boolean)
+ extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf, rapidsConf) {
private val isCaseSensitive = sqlConf.caseSensitiveAnalysis
private val debugDumpPrefix = rapidsConf.parquetDebugDumpPrefix
@@ -1107,15 +1105,11 @@ case class GpuParquetMultiFilePartitionReaderFactory(
"the deprecated one will be honored if both are set.")
deprecatedVal
}.getOrElse(rapidsConf.getMultithreadedReaderKeepOrder)
- private val alluxioReplacementTaskTime =
- AlluxioCfgUtils.enabledAlluxioReplacementAlgoTaskTime(rapidsConf)
private val compressCfg = CpuCompressionConfig.forParquet(rapidsConf)
// We can't use the coalescing files reader when InputFileName, InputFileBlockStart,
// or InputFileBlockLength because we are combining all the files into a single buffer
- // and we don't know which file is associated with each row. If this changes we need to
- // make sure the Alluxio path replacement also handles setting the input file name to
- // the non-Alluxio path like the multi-threaded reader does.
+ // and we don't know which file is associated with each row.
override val canUseCoalesceFilesReader: Boolean =
rapidsConf.isParquetCoalesceFileReadEnabled && !(queryUsesInputFile || ignoreCorruptFiles)
@@ -1143,8 +1137,8 @@ case class GpuParquetMultiFilePartitionReaderFactory(
targetBatchSizeBytes, maxGpuColumnSizeBytes,
useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes, compressCfg,
metrics, partitionSchema, numThreads, maxNumFileProcessed, ignoreMissingFiles,
- ignoreCorruptFiles, readUseFieldId, alluxioPathReplacementMap.getOrElse(Map.empty),
- alluxioReplacementTaskTime, queryUsesInputFile, keepReadsInOrderFromConf, combineConf)
+ ignoreCorruptFiles, readUseFieldId, queryUsesInputFile, keepReadsInOrderFromConf,
+ combineConf)
}
private def filterBlocksForCoalescingReader(
@@ -1209,18 +1203,8 @@ case class GpuParquetMultiFilePartitionReaderFactory(
* @return coalescing reading PartitionReader
*/
override def buildBaseColumnarReaderForCoalescing(
- origFiles: Array[PartitionedFile],
+ files: Array[PartitionedFile],
conf: Configuration): PartitionReader[ColumnarBatch] = {
- // update the file paths for Alluxio if needed, the coalescing reader doesn't support
- // input_file_name so no need to track what the non Alluxio file name is
- val files = if (alluxioReplacementTaskTime) {
- AlluxioUtils.updateFilesTaskTimeIfAlluxio(origFiles, alluxioPathReplacementMap).map(_.toRead)
- } else {
- // Since coalescing reader isn't supported if input_file_name is used, so won't
- // ever get here with that. So with convert time or no Alluxio just use the files as
- // passed in.
- origFiles
- }
val clippedBlocks = ArrayBuffer[ParquetSingleDataBlockMeta]()
val startTime = System.nanoTime()
val metaAndFilesArr = if (numFilesFilterParallel > 0) {
@@ -1289,8 +1273,7 @@ case class GpuParquetPartitionReaderFactory(
filters: Array[Filter],
@transient rapidsConf: RapidsConf,
metrics: Map[String, GpuMetric],
- @transient params: Map[String, String],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ @transient params: Map[String, String])
extends ShimFilePartitionReaderFactory(params) with Logging {
private val isCaseSensitive = sqlConf.caseSensitiveAnalysis
@@ -2336,8 +2319,6 @@ class MultiFileParquetPartitionReader(
* @param ignoreMissingFiles Whether to ignore missing files
* @param ignoreCorruptFiles Whether to ignore corrupt files
* @param useFieldId Whether to use field id for column matching
- * @param alluxioPathReplacementMap Map containing mapping of DFS scheme to Alluxio scheme
- * @param alluxioReplacementTaskTime Whether the Alluxio replacement algorithm is set to task time
* @param queryUsesInputFile Whether the query requires the input file name functionality
* @param keepReadsInOrder Whether to require the files to be read in the same order as Spark.
* Defaults to true for formats that don't explicitly handle this.
@@ -2364,14 +2345,12 @@ class MultiFileCloudParquetPartitionReader(
ignoreMissingFiles: Boolean,
ignoreCorruptFiles: Boolean,
useFieldId: Boolean,
- alluxioPathReplacementMap: Map[String, String],
- alluxioReplacementTaskTime: Boolean,
queryUsesInputFile: Boolean,
keepReadsInOrder: Boolean,
combineConf: CombineConf)
extends MultiFileCloudPartitionReaderBase(conf, files, numThreads, maxNumFileProcessed, null,
execMetrics, maxReadBatchSizeRows, maxReadBatchSizeBytes, ignoreCorruptFiles,
- alluxioPathReplacementMap, alluxioReplacementTaskTime, keepReadsInOrder, combineConf)
+ keepReadsInOrder, combineConf)
with ParquetPartitionReaderBase {
def checkIfNeedToSplit(current: HostMemoryBuffersWithMetaData,
@@ -2489,7 +2468,6 @@ class MultiFileCloudParquetPartitionReader(
combinedMeta.allPartValues.map(_._1).sum, Seq.empty)
val newHmbMeta = HostMemoryBuffersWithMetaData(
metaToUse.partitionedFile,
- metaToUse.origPartitionedFile, // this doesn't matter since already read
Array(newHmbBufferInfo),
offset,
metaToUse.dateRebaseMode,
@@ -2589,7 +2567,6 @@ class MultiFileCloudParquetPartitionReader(
if (combinedEmptyMeta.allEmpty) {
val metaForEmpty = combinedEmptyMeta.metaForEmpty
HostMemoryEmptyMetaData(metaForEmpty.partitionedFile, // just pick one since not used
- metaForEmpty.origPartitionedFile,
combinedEmptyMeta.emptyBufferSize,
combinedEmptyMeta.emptyTotalBytesRead,
metaForEmpty.dateRebaseMode, // these shouldn't matter since data is empty
@@ -2608,7 +2585,6 @@ class MultiFileCloudParquetPartitionReader(
private case class HostMemoryEmptyMetaData(
override val partitionedFile: PartitionedFile,
- override val origPartitionedFile: Option[PartitionedFile],
bufferSize: Long,
override val bytesRead: Long,
dateRebaseMode: DateTimeRebaseMode,
@@ -2625,7 +2601,6 @@ class MultiFileCloudParquetPartitionReader(
case class HostMemoryBuffersWithMetaData(
override val partitionedFile: PartitionedFile,
- override val origPartitionedFile: Option[PartitionedFile],
override val memBuffersAndSizes: Array[SingleHMBAndMeta],
override val bytesRead: Long,
dateRebaseMode: DateTimeRebaseMode,
@@ -2638,7 +2613,6 @@ class MultiFileCloudParquetPartitionReader(
private class ReadBatchRunner(
file: PartitionedFile,
- origPartitionedFile: Option[PartitionedFile],
filterFunc: PartitionedFile => ParquetFileInfoWithBlockMeta,
taskContext: TaskContext) extends Callable[HostMemoryBuffersWithMetaDataBase] with Logging {
@@ -2659,7 +2633,7 @@ class MultiFileCloudParquetPartitionReader(
} catch {
case e: FileNotFoundException if ignoreMissingFiles =>
logWarning(s"Skipped missing file: ${file.filePath}", e)
- HostMemoryEmptyMetaData(file, origPartitionedFile, 0, 0,
+ HostMemoryEmptyMetaData(file, 0, 0,
DateTimeRebaseLegacy, DateTimeRebaseLegacy,
hasInt96Timestamps = false, null, null, 0)
// Throw FileNotFoundException even if `ignoreCorruptFiles` is true
@@ -2667,7 +2641,7 @@ class MultiFileCloudParquetPartitionReader(
case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
logWarning(
s"Skipped the rest of the content in the corrupted file: ${file.filePath}", e)
- HostMemoryEmptyMetaData(file, origPartitionedFile, 0, 0,
+ HostMemoryEmptyMetaData(file, 0, 0,
DateTimeRebaseLegacy, DateTimeRebaseLegacy,
hasInt96Timestamps = false, null, null, 0)
} finally {
@@ -2688,7 +2662,7 @@ class MultiFileCloudParquetPartitionReader(
if (fileBlockMeta.blocks.isEmpty) {
val bytesRead = fileSystemBytesRead() - startingBytesRead
// no blocks so return null buffer and size 0
- HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead,
+ HostMemoryEmptyMetaData(file, 0, bytesRead,
fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode,
fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema, 0)
} else {
@@ -2696,14 +2670,14 @@ class MultiFileCloudParquetPartitionReader(
if (isDone) {
val bytesRead = fileSystemBytesRead() - startingBytesRead
// got close before finishing
- HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead,
+ HostMemoryEmptyMetaData(file, 0, bytesRead,
fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode,
fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema, 0)
} else {
if (fileBlockMeta.schema.getFieldCount == 0) {
val bytesRead = fileSystemBytesRead() - startingBytesRead
val numRows = fileBlockMeta.blocks.map(_.getRowCount).sum.toInt
- HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead,
+ HostMemoryEmptyMetaData(file, 0, bytesRead,
fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode,
fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema,
numRows)
@@ -2722,12 +2696,12 @@ class MultiFileCloudParquetPartitionReader(
if (isDone) {
// got close before finishing
hostBuffers.foreach(_.hmb.safeClose())
- HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead,
+ HostMemoryEmptyMetaData(file, 0, bytesRead,
fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode,
fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema,
fileBlockMeta.readSchema, 0)
} else {
- HostMemoryBuffersWithMetaData(file, origPartitionedFile, hostBuffers.toArray,
+ HostMemoryBuffersWithMetaData(file, hostBuffers.toArray,
bytesRead, fileBlockMeta.dateRebaseMode,
fileBlockMeta.timestampRebaseMode, fileBlockMeta.hasInt96Timestamps,
fileBlockMeta.schema, fileBlockMeta.readSchema, None)
@@ -2758,10 +2732,9 @@ class MultiFileCloudParquetPartitionReader(
override def getBatchRunner(
tc: TaskContext,
file: PartitionedFile,
- origFile: Option[PartitionedFile],
conf: Configuration,
filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = {
- new ReadBatchRunner(file, origFile, filterFunc, tc)
+ new ReadBatchRunner(file, filterFunc, tc)
}
/**
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala
index 46b659c8fcb..ccc6e120d25 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2020-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2020-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -44,8 +44,7 @@ class GpuReadCSVFileFormat extends CSVFileFormat with GpuReadFileFormatWithMetri
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]] = None)
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
val sqlConf = sparkSession.sessionState.conf
val broadcastedHadoopConf =
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala
index 9b384916ab5..09db30759c7 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2020-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2020-2025, NVIDIA CORPORATION.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@@ -63,8 +63,7 @@ trait GpuReadFileFormatWithMetrics extends FileFormat {
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow]
def isPerFileReadEnabled(conf: RapidsConf): Boolean
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala
index 1792f36e09b..940d977c769 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2020-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2020-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -43,8 +43,7 @@ class GpuReadOrcFileFormat extends OrcFileFormat with GpuReadFileFormatWithMetri
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]] = None)
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
val sqlConf = sparkSession.sessionState.conf
val broadcastedHadoopConf =
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala
index e9b8beba036..52f8b0e8d54 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2020-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2020-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -43,8 +43,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]])
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
val sqlConf = sparkSession.sessionState.conf
val broadcastedHadoopConf =
@@ -58,8 +57,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW
filters.toArray,
new RapidsConf(sqlConf),
metrics,
- options,
- alluxioPathReplacementMap)
+ options)
PartitionReaderIterator.buildReader(factory)
}
@@ -78,8 +76,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW
pushedFilters,
fileScan.rapidsConf,
fileScan.allMetrics,
- fileScan.queryUsesInputFile,
- fileScan.alluxioPathsMap)
+ fileScan.queryUsesInputFile)
}
}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index eeeb930cf9a..6c2ef2a0a0d 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -1994,108 +1994,6 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression.
.booleanConf
.createWithDefault(false)
- // ALLUXIO CONFIGS
- val ALLUXIO_MASTER = conf("spark.rapids.alluxio.master")
- .doc("The Alluxio master hostname. If not set, read Alluxio master URL from " +
- "spark.rapids.alluxio.home locally. This config is useful when Alluxio master " +
- "and Spark driver are not co-located.")
- .startupOnly()
- .stringConf
- .createWithDefault("")
-
- val ALLUXIO_MASTER_PORT = conf("spark.rapids.alluxio.master.port")
- .doc("The Alluxio master port. If not set, read Alluxio master port from " +
- "spark.rapids.alluxio.home locally. This config is useful when Alluxio master " +
- "and Spark driver are not co-located.")
- .startupOnly()
- .integerConf
- .createWithDefault(19998)
-
- val ALLUXIO_HOME = conf("spark.rapids.alluxio.home")
- .doc("The Alluxio installation home path or link to the installation home path. ")
- .startupOnly()
- .stringConf
- .createWithDefault("/opt/alluxio")
-
- val ALLUXIO_PATHS_REPLACE = conf("spark.rapids.alluxio.pathsToReplace")
- .doc("List of paths to be replaced with corresponding Alluxio scheme. " +
- "E.g. when configure is set to " +
- "\"s3://foo->alluxio://0.1.2.3:19998/foo,gs://bar->alluxio://0.1.2.3:19998/bar\", " +
- "it means: " +
- "\"s3://foo/a.csv\" will be replaced to \"alluxio://0.1.2.3:19998/foo/a.csv\" and " +
- "\"gs://bar/b.csv\" will be replaced to \"alluxio://0.1.2.3:19998/bar/b.csv\". " +
- "To use this config, you have to mount the buckets to Alluxio by yourself. " +
- "If you set this config, spark.rapids.alluxio.automount.enabled won't be valid.")
- .startupOnly()
- .stringConf
- .toSequence
- .createOptional
-
- val ALLUXIO_AUTOMOUNT_ENABLED = conf("spark.rapids.alluxio.automount.enabled")
- .doc("Enable the feature of auto mounting the cloud storage to Alluxio. " +
- "It requires the Alluxio master is the same node of Spark driver node. " +
- "The Alluxio master's host and port will be read from alluxio.master.hostname and " +
- "alluxio.master.rpc.port(default: 19998) from ALLUXIO_HOME/conf/alluxio-site.properties, " +
- "then replace a cloud path which matches spark.rapids.alluxio.bucket.regex like " +
- "\"s3://bar/b.csv\" to \"alluxio://0.1.2.3:19998/bar/b.csv\", " +
- "and the bucket \"s3://bar\" will be mounted to \"/bar\" in Alluxio automatically.")
- .booleanConf
- .createWithDefault(false)
-
- val ALLUXIO_BUCKET_REGEX = conf("spark.rapids.alluxio.bucket.regex")
- .doc("A regex to decide which bucket should be auto-mounted to Alluxio. " +
- "E.g. when setting as \"^s3://bucket.*\", " +
- "the bucket which starts with \"s3://bucket\" will be mounted to Alluxio " +
- "and the path \"s3://bucket-foo/a.csv\" will be replaced to " +
- "\"alluxio://0.1.2.3:19998/bucket-foo/a.csv\". " +
- "It's only valid when setting spark.rapids.alluxio.automount.enabled=true. " +
- "The default value matches all the buckets in \"s3://\" or \"s3a://\" scheme.")
- .stringConf
- .createWithDefault("^s3a{0,1}://.*")
-
- val ALLUXIO_USER = conf("spark.rapids.alluxio.user")
- .doc("Alluxio user is set on the Alluxio client, " +
- "which is used to mount or get information. " +
- "By default it should be the user that running the Alluxio processes. " +
- "The default value is ubuntu.")
- .stringConf
- .createWithDefault("ubuntu")
-
- val ALLUXIO_REPLACEMENT_ALGO = conf("spark.rapids.alluxio.replacement.algo")
- .doc("The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME " +
- "and TASK_TIME are the valid options. CONVERT_TIME indicates that we do it " +
- "when we convert it to a GPU file read, this has extra overhead of creating an entirely " +
- "new file index, which requires listing the files and getting all new file info from " +
- "Alluxio. TASK_TIME replaces the path as late as possible inside of the task. " +
- "By waiting and replacing it at task time, it just replaces " +
- "the path without fetching the file information again, this is faster " +
- "but doesn't update locality information if that has a bit impact on performance.")
- .stringConf
- .checkValues(Set("CONVERT_TIME", "TASK_TIME"))
- .createWithDefault("TASK_TIME")
-
- val ALLUXIO_LARGE_FILE_THRESHOLD = conf("spark.rapids.alluxio.large.file.threshold")
- .doc("The threshold is used to identify whether average size of files is large " +
- "when reading from S3. If reading large files from S3 and " +
- "the disks used by Alluxio are slow, " +
- "directly reading from S3 is better than reading caches from Alluxio, " +
- "because S3 network bandwidth is faster than local disk. " +
- "This improvement takes effect when spark.rapids.alluxio.slow.disk is enabled.")
- .bytesConf(ByteUnit.BYTE)
- .createWithDefault(64 * 1024 * 1024) // 64M
-
- val ALLUXIO_SLOW_DISK = conf("spark.rapids.alluxio.slow.disk")
- .doc("Indicates whether the disks used by Alluxio are slow. " +
- "If it's true and reading S3 large files, " +
- "Rapids Accelerator reads from S3 directly instead of reading from Alluxio caches. " +
- "Refer to spark.rapids.alluxio.large.file.threshold which defines a threshold that " +
- "identifying whether files are large. " +
- "Typically, it's slow disks if speed is less than 300M/second. " +
- "If using convert time spark.rapids.alluxio.replacement.algo, " +
- "this may not apply to all file types like Delta files")
- .booleanConf
- .createWithDefault(true)
-
// USER FACING DEBUG CONFIGS
val SHUFFLE_COMPRESSION_MAX_BATCH_MEMORY =
@@ -2294,8 +2192,7 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression.
"the files will be filtered in a multithreaded manner where each thread filters " +
"the number of files set by this config. If this is set to zero the files are " +
"filtered serially. This uses the same thread pool as the multithreaded reader, " +
- s"see $MULTITHREAD_READ_NUM_THREADS. Note that filtering multithreaded " +
- "is useful with Alluxio.")
+ s"see $MULTITHREAD_READ_NUM_THREADS.")
.integerConf
.createWithDefault(value = 0)
@@ -3240,32 +3137,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging {
lazy val gpuWriteMemorySpeed: Double = get(OPTIMIZER_GPU_WRITE_SPEED)
- lazy val getAlluxioHome: String = get(ALLUXIO_HOME)
-
- lazy val getAlluxioMaster: String = get(ALLUXIO_MASTER)
-
- lazy val getAlluxioMasterPort: Int = get(ALLUXIO_MASTER_PORT)
-
- lazy val getAlluxioPathsToReplace: Option[Seq[String]] = get(ALLUXIO_PATHS_REPLACE)
-
- lazy val getAlluxioAutoMountEnabled: Boolean = get(ALLUXIO_AUTOMOUNT_ENABLED)
-
- lazy val getAlluxioBucketRegex: String = get(ALLUXIO_BUCKET_REGEX)
-
- lazy val getAlluxioUser: String = get(ALLUXIO_USER)
-
- lazy val getAlluxioReplacementAlgo: String = get(ALLUXIO_REPLACEMENT_ALGO)
-
- lazy val isAlluxioReplacementAlgoConvertTime: Boolean =
- get(ALLUXIO_REPLACEMENT_ALGO) == "CONVERT_TIME"
-
- lazy val isAlluxioReplacementAlgoTaskTime: Boolean =
- get(ALLUXIO_REPLACEMENT_ALGO) == "TASK_TIME"
-
- lazy val getAlluxioLargeFileThreshold: Long = get(ALLUXIO_LARGE_FILE_THRESHOLD)
-
- lazy val enableAlluxioSlowDisk: Boolean = get(ALLUXIO_SLOW_DISK)
-
lazy val driverTimeZone: Option[String] = get(DRIVER_TIMEZONE)
lazy val isRangeWindowByteEnabled: Boolean = get(ENABLE_RANGE_WINDOW_BYTES)
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala
index b7b7d63c3cb..68a68da2073 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2022-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -46,8 +46,7 @@ class GpuReadJsonFileFormat extends JsonFileFormat with GpuReadFileFormatWithMet
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]] = None)
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
val sqlConf = sparkSession.sessionState.conf
val broadcastedHadoopConf =
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala
deleted file mode 100644
index 83ae5fb2afd..00000000000
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala
+++ /dev/null
@@ -1,627 +0,0 @@
-/*
- * Copyright (c) 2021-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.
- */
-
-package org.apache.spark.sql.execution.datasources.rapids
-
-import java.lang.{Double => JDouble, Long => JLong}
-import java.math.{BigDecimal => JBigDecimal}
-import java.time.ZoneId
-import java.util.Locale
-
-import scala.collection.mutable.ArrayBuffer
-import scala.util.Try
-import scala.util.control.NonFatal
-
-import com.nvidia.spark.rapids.AlluxioUtils.AlluxioPathReplaceConvertTime
-import com.nvidia.spark.rapids.shims.FileIndexOptionsShims.BASE_PATH_PARAM
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper}
-import org.apache.spark.sql.catalyst.analysis.TypeCoercion
-import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.{unescapePathName, DEFAULT_PARTITION_NAME}
-import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
-import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils, TimestampFormatter}
-import org.apache.spark.sql.catalyst.util.rapids.DateFormatter
-import org.apache.spark.sql.execution.datasources.{PartitionPath, PartitionSpec}
-import org.apache.spark.sql.execution.datasources.PartitioningUtils.timestampPartitionPattern
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.types.shims.PartitionValueCastShims
-import org.apache.spark.unsafe.types.UTF8String
-
-object GpuPartitioningUtils extends SQLConfHelper {
-
- case class TypedPartValue(value: String, dataType: DataType)
-
- case class PartitionValues(columnNames: Seq[String], typedValues: Seq[TypedPartValue])
-
- /**
- *
- * @param sparkSession
- * @param rootPaths the list of root input paths from which the catalog will get files
- * @param leafFiles leaf file paths
- * @param parameters a set of options to control partition discovery
- * @param userSpecifiedSchema an optional user specified schema that will be use to provide
- * types for the discovered partitions
- * @param replaceFunc the alluxio replace function
- * @return the specification of the partitions inferred from the data and if it was replaced the
- * original path
- *
- * Mainly copied from PartitioningAwareFileIndex.inferPartitioning
- */
- def inferPartitioning(
- sparkSession: SparkSession,
- rootPaths: Seq[Path],
- leafFiles: Seq[Path],
- parameters: Map[String, String],
- userSpecifiedSchema: Option[StructType],
- replaceFunc: Path => AlluxioPathReplaceConvertTime)
- : (PartitionSpec, Option[String]) = {
-
- val recursiveFileLookup = parameters.getOrElse("recursiveFileLookup", "false").toBoolean
-
- if (recursiveFileLookup) {
- (PartitionSpec.emptySpec, None)
- } else {
- val caseInsensitiveOptions = CaseInsensitiveMap(parameters)
- val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION)
- .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone)
-
- // filter out non-data path and get unique leaf dirs of inputFiles
- val leafDirs: Seq[Path] = leafFiles.filter(isDataPath).map(_.getParent).distinct
-
- val basePathAndAnyReplacedOption = parameters.get(BASE_PATH_PARAM).map { file =>
- // need to replace the base path
- replaceFunc(new Path(file))
- }
- val basePathOption = basePathAndAnyReplacedOption.map(_.filePath)
- val anyReplacedBase = basePathAndAnyReplacedOption.flatMap(_.origPrefix)
-
- val basePaths = getBasePaths(sparkSession.sessionState.newHadoopConfWithOptions(parameters),
- basePathOption, rootPaths, leafFiles)
-
- val parsed = parsePartitions(
- leafDirs,
- typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled,
- basePaths = basePaths,
- userSpecifiedSchema = userSpecifiedSchema,
- caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis,
- validatePartitionColumns = sparkSession.sessionState.conf.validatePartitionColumns,
- timeZoneId = timeZoneId)
- (parsed, anyReplacedBase)
- }
- }
-
- // SPARK-15895: Metadata files (e.g. Parquet summary files) and temporary files should not be
- // counted as data files, so that they shouldn't participate partition discovery.
- // Copied from PartitioningAwareFileIndex.isDataPath
- def isDataPath(path: Path): Boolean = {
- isDataPath(path.getName)
- }
-
- def isDataPath(name: String): Boolean = {
- !((name.startsWith("_") && !name.contains("=")) || name.startsWith("."))
- }
-
- /**
- * Contains a set of paths that are considered as the base dirs of the input datasets.
- * The partitioning discovery logic will make sure it will stop when it reaches any
- * base path.
- *
- * By default, the paths of the dataset provided by users will be base paths.
- * Below are three typical examples,
- * Case 1) `spark.read.parquet("/path/something=true/")`: the base path will be
- * `/path/something=true/`, and the returned DataFrame will not contain a column of `something`.
- * Case 2) `spark.read.parquet("/path/something=true/a.parquet")`: the base path will be
- * still `/path/something=true/`, and the returned DataFrame will also not contain a column of
- * `something`.
- * Case 3) `spark.read.parquet("/path/")`: the base path will be `/path/`, and the returned
- * DataFrame will have the column of `something`.
- *
- * Users also can override the basePath by setting `basePath` in the options to pass the new base
- * path to the data source.
- * For example, `spark.read.option("basePath", "/path/").parquet("/path/something=true/")`,
- * and the returned DataFrame will have the column of `something`.
- *
- * mainly copied from PartitioningAwareFileIndex.basePaths
- */
- private def getBasePaths(
- hadoopConf: Configuration,
- basePathOption: Option[Path],
- rootPaths: Seq[Path],
- leafFiles: Seq[Path]): Set[Path] = {
- basePathOption match {
- case Some(userDefinedBasePath) =>
- val fs = userDefinedBasePath.getFileSystem(hadoopConf)
- if (!fs.getFileStatus(userDefinedBasePath).isDirectory) {
- throw new IllegalArgumentException(s"Option '$BASE_PATH_PARAM' must be a directory")
- }
- val qualifiedBasePath = fs.makeQualified(userDefinedBasePath)
- val qualifiedBasePathStr = qualifiedBasePath.toString
- rootPaths
- .find(!fs.makeQualified(_).toString.startsWith(qualifiedBasePathStr))
- .foreach { rp =>
- throw new IllegalArgumentException(
- s"Wrong basePath $userDefinedBasePath for the root path: $rp")
- }
- Set(qualifiedBasePath)
-
- case None =>
- rootPaths.map { path =>
- // Make the path qualified (consistent with listLeafFiles and bulkListLeafFiles).
- val qualifiedPath = path.getFileSystem(hadoopConf).makeQualified(path)
- if (leafFiles.contains(qualifiedPath)) qualifiedPath.getParent else qualifiedPath }.toSet
- }
- }
-
- // To fix issue: https://github.com/NVIDIA/spark-rapids/issues/6026 and
- // https://issues.apache.org/jira/browse/SPARK-39012
- // Port latest Spark code into spark-rapids
- def castPartValueToDesiredType(
- desiredType: DataType,
- value: String,
- zoneId: ZoneId): Any = desiredType match {
- case _ if value == DEFAULT_PARTITION_NAME => null
- case NullType => null
- case StringType => UTF8String.fromString(unescapePathName(value))
- case ByteType => Integer.parseInt(value).toByte
- case ShortType => Integer.parseInt(value).toShort
- case IntegerType => Integer.parseInt(value)
- case LongType => JLong.parseLong(value)
- case FloatType => JDouble.parseDouble(value).toFloat
- case DoubleType => JDouble.parseDouble(value)
- case _: DecimalType => Literal(new JBigDecimal(value)).value
- case DateType =>
- Cast(Literal(value), DateType, Some(zoneId.getId)).eval()
- case BinaryType => value.getBytes()
- case BooleanType => value.toBoolean
- case t if PartitionValueCastShims.isSupportedType(t) =>
- PartitionValueCastShims.castTo(t, value, zoneId)
- case dt => throw new IllegalArgumentException(s"Unexpected type $dt")
- }
-
- /**
- * Given a group of qualified paths, tries to parse them and returns a partition specification.
- * For example, given:
- * {{{
- * hdfs://:/path/to/partition/a=1/b=hello/c=3.14
- * hdfs://:/path/to/partition/a=2/b=world/c=6.28
- * }}}
- * it returns:
- * {{{
- * PartitionSpec(
- * partitionColumns = StructType(
- * StructField(name = "a", dataType = IntegerType, nullable = true),
- * StructField(name = "b", dataType = StringType, nullable = true),
- * StructField(name = "c", dataType = DoubleType, nullable = true)),
- * partitions = Seq(
- * Partition(
- * values = Row(1, "hello", 3.14),
- * path = "hdfs://:/path/to/partition/a=1/b=hello/c=3.14"),
- * Partition(
- * values = Row(2, "world", 6.28),
- * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28")))
- * }}}
- */
- private[datasources] def parsePartitions(
- paths: Seq[Path],
- typeInference: Boolean,
- basePaths: Set[Path],
- userSpecifiedSchema: Option[StructType],
- caseSensitive: Boolean,
- validatePartitionColumns: Boolean,
- timeZoneId: String): PartitionSpec = {
- parsePartitions(paths, typeInference, basePaths, userSpecifiedSchema, caseSensitive,
- validatePartitionColumns, DateTimeUtils.getZoneId(timeZoneId))
- }
-
- private[datasources] def parsePartitions(
- paths: Seq[Path],
- typeInference: Boolean,
- basePaths: Set[Path],
- userSpecifiedSchema: Option[StructType],
- caseSensitive: Boolean,
- validatePartitionColumns: Boolean,
- zoneId: ZoneId): PartitionSpec = {
- val userSpecifiedDataTypes = if (userSpecifiedSchema.isDefined) {
- val nameToDataType = userSpecifiedSchema.get.fields.map(f => f.name -> f.dataType).toMap
- if (!caseSensitive) {
- CaseInsensitiveMap(nameToDataType)
- } else {
- nameToDataType
- }
- } else {
- Map.empty[String, DataType]
- }
-
- // SPARK-26990: use user specified field names if case insensitive.
- val userSpecifiedNames = if (userSpecifiedSchema.isDefined && !caseSensitive) {
- CaseInsensitiveMap(userSpecifiedSchema.get.fields.map(f => f.name -> f.name).toMap)
- } else {
- Map.empty[String, String]
- }
-
- val dateFormatter = DateFormatter(DateFormatter.defaultPattern)
- val timestampFormatter = TimestampFormatter(
- timestampPartitionPattern,
- zoneId,
- isParsing = true)
- // First, we need to parse every partition's path and see if we can find partition values.
- val (partitionValues, optDiscoveredBasePaths) = paths.map { path =>
- parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes,
- validatePartitionColumns, zoneId, dateFormatter, timestampFormatter)
- }.unzip
-
- // We create pairs of (path -> path's partition value) here
- // If the corresponding partition value is None, the pair will be skipped
- val pathsWithPartitionValues = paths.zip(partitionValues).flatMap(x => x._2.map(x._1 -> _))
-
- if (pathsWithPartitionValues.isEmpty) {
- // This dataset is not partitioned.
- PartitionSpec.emptySpec
- } else {
- // This dataset is partitioned. We need to check whether all partitions have the same
- // partition columns and resolve potential type conflicts.
-
- // Check if there is conflicting directory structure.
- // For the paths such as:
- // var paths = Seq(
- // "hdfs://host:9000/invalidPath",
- // "hdfs://host:9000/path/a=10/b=20",
- // "hdfs://host:9000/path/a=10.5/b=hello")
- // It will be recognised as conflicting directory structure:
- // "hdfs://host:9000/invalidPath"
- // "hdfs://host:9000/path"
- // TODO: Selective case sensitivity.
- val discoveredBasePaths = optDiscoveredBasePaths.flatten.map(_.toString.toLowerCase())
- assert(
- discoveredBasePaths.distinct.size == 1,
- "Conflicting directory structures detected. Suspicious paths:\b" +
- discoveredBasePaths.distinct.mkString("\n\t", "\n\t", "\n\n") +
- "If provided paths are partition directories, please set " +
- "\"basePath\" in the options of the data source to specify the " +
- "root directory of the table. If there are multiple root directories, " +
- "please load them separately and then union them.")
-
- val resolvedPartitionValues = resolvePartitions(pathsWithPartitionValues, caseSensitive)
-
- // Creates the StructType which represents the partition columns.
- val fields = {
- val PartitionValues(columnNames, typedValues) = resolvedPartitionValues.head
- columnNames.zip(typedValues).map { case (name, TypedPartValue(_, dataType)) =>
- // We always assume partition columns are nullable since we've no idea whether null values
- // will be appended in the future.
- val resultName = userSpecifiedNames.getOrElse(name, name)
- val resultDataType = userSpecifiedDataTypes.getOrElse(name, dataType)
- StructField(resultName, resultDataType, nullable = true)
- }
- }
-
- // Finally, we create `Partition`s based on paths and resolved partition values.
- val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map {
- case (PartitionValues(columnNames, typedValues), (path, _)) =>
- val rowValues = columnNames.zip(typedValues).map { case (columnName, typedValue) =>
- try {
- castPartValueToDesiredType(typedValue.dataType, typedValue.value, zoneId)
- } catch {
- case NonFatal(_) =>
- if (validatePartitionColumns) {
- throw new RuntimeException(s"Failed to cast value `$typedValue.value` to " +
- s"`$typedValue.dataType` for partition column `$columnName`")
- } else null
- }
- }
- PartitionPath(InternalRow.fromSeq(rowValues), path)
- }
-
- PartitionSpec(StructType(fields), partitions)
- }
- }
-
- /**
- * Parses a single partition, returns column names and values of each partition column, also
- * the path when we stop partition discovery. For example, given:
- * {{{
- * path = hdfs://:/path/to/partition/a=42/b=hello/c=3.14
- * }}}
- * it returns the partition:
- * {{{
- * PartitionValues(
- * Seq("a", "b", "c"),
- * Seq(
- * Literal.create(42, IntegerType),
- * Literal.create("hello", StringType),
- * Literal.create(3.14, DoubleType)))
- * }}}
- * and the path when we stop the discovery is:
- * {{{
- * hdfs://:/path/to/partition
- * }}}
- */
- private[datasources] def parsePartition(
- path: Path,
- typeInference: Boolean,
- basePaths: Set[Path],
- userSpecifiedDataTypes: Map[String, DataType],
- validatePartitionColumns: Boolean,
- zoneId: ZoneId,
- dateFormatter: DateFormatter,
- timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = {
- val columns = ArrayBuffer.empty[(String, TypedPartValue)]
- // Old Hadoop versions don't have `Path.isRoot`
- var finished = path.getParent == null
- // currentPath is the current path that we will use to parse partition column value.
- var currentPath: Path = path
-
- while (!finished) {
- // Sometimes (e.g., when speculative task is enabled), temporary directories may be left
- // uncleaned. Here we simply ignore them.
- if (currentPath.getName.toLowerCase(Locale.ROOT) == "_temporary") {
- return (None, None)
- }
-
- if (basePaths.contains(currentPath)) {
- // If the currentPath is one of base paths. We should stop.
- finished = true
- } else {
- // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1.
- // Once we get the string, we try to parse it and find the partition column and value.
- val maybeColumn =
- parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes,
- zoneId, dateFormatter, timestampFormatter)
- maybeColumn.foreach(columns += _)
-
- // Now, we determine if we should stop.
- // When we hit any of the following cases, we will stop:
- // - In this iteration, we could not parse the value of partition column and value,
- // i.e. maybeColumn is None, and columns is not empty. At here we check if columns is
- // empty to handle cases like /table/a=1/_temporary/something (we need to find a=1 in
- // this case).
- // - After we get the new currentPath, this new currentPath represent the top level dir
- // i.e. currentPath.getParent == null. For the example of "/table/a=1/",
- // the top level dir is "/table".
- finished =
- (maybeColumn.isEmpty && !columns.isEmpty) || currentPath.getParent == null
-
- if (!finished) {
- // For the above example, currentPath will be "/table/".
- currentPath = currentPath.getParent
- }
- }
- }
-
- if (columns.isEmpty) {
- (None, Some(path))
- } else {
- val (columnNames, values) = columns.reverse.unzip
- (Some(PartitionValues(columnNames.toSeq, values.toSeq)), Some(currentPath))
- }
- }
-
- private def parsePartitionColumn(
- columnSpec: String,
- typeInference: Boolean,
- userSpecifiedDataTypes: Map[String, DataType],
- zoneId: ZoneId,
- dateFormatter: DateFormatter,
- timestampFormatter: TimestampFormatter): Option[(String, TypedPartValue)] = {
- val equalSignIndex = columnSpec.indexOf('=')
- if (equalSignIndex == -1) {
- None
- } else {
- val columnName = unescapePathName(columnSpec.take(equalSignIndex))
- assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'")
-
- val rawColumnValue = columnSpec.drop(equalSignIndex + 1)
- assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'")
-
- val dataType = if (userSpecifiedDataTypes.contains(columnName)) {
- // SPARK-26188: if user provides corresponding column schema, get the column value without
- // inference, and then cast it as user specified data type.
- userSpecifiedDataTypes(columnName)
- } else {
- inferPartitionColumnValue(
- rawColumnValue,
- typeInference,
- zoneId,
- dateFormatter,
- timestampFormatter)
- }
- Some(columnName -> TypedPartValue(rawColumnValue, dataType))
- }
- }
-
- private[datasources] def listConflictingPartitionColumns(
- pathWithPartitionValues: Seq[(Path, PartitionValues)]): String = {
- val distinctPartColNames = pathWithPartitionValues.map(_._2.columnNames).distinct
-
- def groupByKey[K, V](seq: Seq[(K, V)]): Map[K, Iterable[V]] =
- seq.groupBy { case (key, _) => key }.mapValues(_.map { case (_, value) => value }).toMap
-
- val partColNamesToPaths = groupByKey(pathWithPartitionValues.map {
- case (path, partValues) => partValues.columnNames -> path
- })
-
- val distinctPartColLists = distinctPartColNames.map(_.mkString(", ")).zipWithIndex.map {
- case (names, index) =>
- s"Partition column name list #$index: $names"
- }
-
- // Lists out those non-leaf partition directories that also contain files
- val suspiciousPaths = distinctPartColNames.sortBy(_.length).flatMap(partColNamesToPaths)
-
- s"Conflicting partition column names detected:\n" +
- distinctPartColLists.mkString("\n\t", "\n\t", "\n\n") +
- "For partitioned table directories, data files should only live in leaf directories.\n" +
- "And directories at the same level should have the same partition column name.\n" +
- "Please check the following directories for unexpected files or " +
- "inconsistent partition column names:\n" +
- suspiciousPaths.map("\t" + _).mkString("\n", "\n", "")
- }
-
- // scalastyle:off line.size.limit
- /**
- * Converts a string to a [[Literal]] with automatic type inference. Currently only supports
- * [[NullType]], [[IntegerType]], [[LongType]], [[DoubleType]], [[DecimalType]], [[DateType]]
- * [[TimestampType]], and [[StringType]].
- *
- * When resolving conflicts, it follows the table below:
- *
- * +--------------------+-------------------+-------------------+-------------------+--------------------+------------+---------------+---------------+------------+
- * | InputA \ InputB | NullType | IntegerType | LongType | DecimalType(38,0)* | DoubleType | DateType | TimestampType | StringType |
- * +--------------------+-------------------+-------------------+-------------------+--------------------+------------+---------------+---------------+------------+
- * | NullType | NullType | IntegerType | LongType | DecimalType(38,0) | DoubleType | DateType | TimestampType | StringType |
- * | IntegerType | IntegerType | IntegerType | LongType | DecimalType(38,0) | DoubleType | StringType | StringType | StringType |
- * | LongType | LongType | LongType | LongType | DecimalType(38,0) | StringType | StringType | StringType | StringType |
- * | DecimalType(38,0)* | DecimalType(38,0) | DecimalType(38,0) | DecimalType(38,0) | DecimalType(38,0) | StringType | StringType | StringType | StringType |
- * | DoubleType | DoubleType | DoubleType | StringType | StringType | DoubleType | StringType | StringType | StringType |
- * | DateType | DateType | StringType | StringType | StringType | StringType | DateType | TimestampType | StringType |
- * | TimestampType | TimestampType | StringType | StringType | StringType | StringType | TimestampType | TimestampType | StringType |
- * | StringType | StringType | StringType | StringType | StringType | StringType | StringType | StringType | StringType |
- * +--------------------+-------------------+-------------------+-------------------+--------------------+------------+---------------+---------------+------------+
- * Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other
- * combinations of scales and precisions because currently we only infer decimal type like
- * `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type.
- */
- // scalastyle:on line.size.limit
- private[datasources] def inferPartitionColumnValue(
- raw: String,
- typeInference: Boolean,
- zoneId: ZoneId,
- dateFormatter: DateFormatter,
- timestampFormatter: TimestampFormatter): DataType = {
- val decimalTry = Try {
- // `BigDecimal` conversion can fail when the `field` is not a form of number.
- val bigDecimal = new JBigDecimal(raw)
- // It reduces the cases for decimals by disallowing values having scale (e.g. `1.1`).
- require(bigDecimal.scale <= 0)
- // `DecimalType` conversion can fail when
- // 1. The precision is bigger than 38.
- // 2. scale is bigger than precision.
- DecimalType.fromDecimal(Decimal(bigDecimal))
- }
-
- val dateTry = Try {
- // try and parse the date, if no exception occurs this is a candidate to be resolved as
- // DateType
- dateFormatter.parse(raw)
- // SPARK-23436: Casting the string to date may still return null if a bad Date is provided.
- // This can happen since DateFormat.parse may not use the entire text of the given string:
- // so if there are extra-characters after the date, it returns correctly.
- // We need to check that we can cast the raw string since we later can use Cast to get
- // the partition values with the right DataType (see
- // org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning)
- val dateValue = Cast(Literal(raw), DateType, Some(zoneId.getId)).eval()
- // Disallow DateType if the cast returned null
- require(dateValue != null)
- DateType
- }
-
- val timestampTry = Try {
- val unescapedRaw = unescapePathName(raw)
- // the inferred data type is consistent with the default timestamp type
- val timestampType = TimestampType
- // try and parse the date, if no exception occurs this is a candidate to be resolved as
- // TimestampType or TimestampNTZType
- timestampType match {
- case TimestampType => timestampFormatter.parse(unescapedRaw)
- // comment the NTZ type for very limited use case of "SPARK_TESTING" to reduce duplicated
- // code copied from Spark
- // case TimestampNTZType => timestampFormatter.parseWithoutTimeZone(unescapedRaw)
- }
- // SPARK-23436: see comment for date
- val timestampValue = Cast(Literal(unescapedRaw), timestampType, Some(zoneId.getId)).eval()
- // Disallow TimestampType if the cast returned null
- require(timestampValue != null)
- timestampType
- }
-
- if (typeInference) {
- // First tries integral types
- Try({ Integer.parseInt(raw); IntegerType })
- .orElse(Try { JLong.parseLong(raw); LongType })
- .orElse(decimalTry)
- // Then falls back to fractional types
- .orElse(Try { JDouble.parseDouble(raw); DoubleType })
- // Then falls back to date/timestamp types
- .orElse(timestampTry)
- .orElse(dateTry)
- // Then falls back to string
- .getOrElse {
- if (raw == DEFAULT_PARTITION_NAME) NullType else StringType
- }
- } else {
- if (raw == DEFAULT_PARTITION_NAME) NullType else StringType
- }
- }
-
- /**
- * Resolves possible type conflicts between partitions by up-casting "lower" types using
- * [[findWiderTypeForPartitionColumn]].
- */
- def resolvePartitions(
- pathsWithPartitionValues: Seq[(Path, PartitionValues)],
- caseSensitive: Boolean): Seq[PartitionValues] = {
- if (pathsWithPartitionValues.isEmpty) {
- Seq.empty
- } else {
- val partColNames = if (caseSensitive) {
- pathsWithPartitionValues.map(_._2.columnNames)
- } else {
- pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase()))
- }
- assert(
- partColNames.distinct.size == 1,
- listConflictingPartitionColumns(pathsWithPartitionValues))
-
- // Resolves possible type conflicts for each column
- val values = pathsWithPartitionValues.map(_._2)
- val columnCount = values.head.columnNames.size
- val resolvedValues = (0 until columnCount).map { i =>
- resolveTypeConflicts(values.map(_.typedValues(i)))
- }
-
- // Fills resolved literals back to each partition
- values.zipWithIndex.map { case (d, index) =>
- d.copy(typedValues = resolvedValues.map(_(index)))
- }
- }
- }
-
- /**
- * Given a collection of [[Literal]]s, resolves possible type conflicts by
- * [[findWiderTypeForPartitionColumn]].
- */
- private def resolveTypeConflicts(typedValues: Seq[TypedPartValue]): Seq[TypedPartValue] = {
- val dataTypes = typedValues.map(_.dataType)
- val desiredType = dataTypes.reduce(findWiderTypeForPartitionColumn)
-
- typedValues.map(tv => tv.copy(dataType = desiredType))
- }
-
- /**
- * Type widening rule for partition column types. It is similar to
- * [[TypeCoercion.findWiderTypeForTwo]] but the main difference is that here we disallow
- * precision loss when widening double/long and decimal, and fall back to string.
- */
- private val findWiderTypeForPartitionColumn: (DataType, DataType) => DataType = {
- case (DoubleType, _: DecimalType) | (_: DecimalType, DoubleType) => StringType
- case (DoubleType, LongType) | (LongType, DoubleType) => StringType
- case (t1, t2) => TypeCoercion.findWiderTypeForTwo(t1, t2).getOrElse(StringType)
- }
-
-}
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala
index 02153b5cb87..e81e2eb9821 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2022-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -700,7 +700,6 @@ class GpuMultiFileCloudAvroPartitionReader(
override def getBatchRunner(
tc: TaskContext,
file: PartitionedFile,
- origFile: Option[PartitionedFile],
config: Configuration,
filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] =
new ReadBatchRunner(tc, file, config, filters)
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala
index 38306991db3..9f280e02c7b 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2020-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2020-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -63,7 +63,6 @@ import org.apache.spark.util.collection.BitSet
* off in GpuTransitionOverrides if InputFileName,
* InputFileBlockStart, or InputFileBlockLength are used
* @param disableBucketedScan Disable bucketed scan based on physical query plan.
- * @param alluxioPathsMap Map containing mapping of DFS scheme to Alluxio scheme
*/
case class GpuFileSourceScanExec(
@transient relation: HadoopFsRelation,
@@ -76,7 +75,6 @@ case class GpuFileSourceScanExec(
tableIdentifier: Option[TableIdentifier],
disableBucketedScan: Boolean = false,
queryUsesInputFile: Boolean = false,
- alluxioPathsMap: Option[Map[String, String]],
requiredPartitionSchema: Option[StructType] = None)(@transient val rapidsConf: RapidsConf)
extends GpuDataSourceScanExec with GpuExec {
import GpuMetric._
@@ -92,12 +90,6 @@ case class GpuFileSourceScanExec(
val readPartitionSchema = requiredPartitionSchema.getOrElse(relation.partitionSchema)
- // this is set only when we either explicitly replaced a path for CONVERT_TIME
- // or when TASK_TIME if one of the paths will be replaced.
- // If reading large s3 files on a cluster with slower disks,
- // should update this to None and read directly from s3 to get faster.
- private var alluxioPathReplacementMap: Option[Map[String, String]] = alluxioPathsMap
-
@transient private val gpuFormat = relation.fileFormat match {
case g: GpuReadFileFormatWithMetrics => g
case f => throw new IllegalStateException(s"${f.getClass} is not a GPU format with metrics")
@@ -135,31 +127,7 @@ case class GpuFileSourceScanExec(
val startTime = System.nanoTime()
val pds = relation.location.listFiles(
partitionFilters.filterNot(isDynamicPruningFilter), dataFilters)
- if (AlluxioCfgUtils.isAlluxioPathsToReplaceTaskTime(rapidsConf, relation.fileFormat)) {
- // if should directly read from s3, should set `alluxioPathReplacementMap` as None
- if (AlluxioUtils.shouldReadDirectlyFromS3(rapidsConf, pds)) {
- alluxioPathReplacementMap = None
- } else {
- // this is not ideal, here we check to see if we will replace any paths, which is an
- // extra iteration through paths
- alluxioPathReplacementMap = AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds,
- relation.sparkSession.sparkContext.hadoopConfiguration,
- relation.sparkSession.conf)
- }
- } else if (AlluxioCfgUtils.isAlluxioAutoMountTaskTime(rapidsConf, relation.fileFormat)) {
- // if should directly read from s3, should set `alluxioPathReplacementMap` as None
- if (AlluxioUtils.shouldReadDirectlyFromS3(rapidsConf, pds)) {
- alluxioPathReplacementMap = None
- } else {
- alluxioPathReplacementMap = AlluxioUtils.autoMountIfNeeded(rapidsConf, pds,
- relation.sparkSession.sparkContext.hadoopConfiguration,
- relation.sparkSession.conf)
- }
- }
-
- logDebug(s"File listing and possibly replace with Alluxio path " +
- s"took: ${System.nanoTime() - startTime}")
-
+ logDebug(s"File listing took: ${System.nanoTime() - startTime}")
setFilesNumAndSizeMetric(pds, true)
val timeTakenMs = NANOSECONDS.toMillis(
(System.nanoTime() - startTime) + optimizerMetadataTimeNs)
@@ -369,8 +337,7 @@ case class GpuFileSourceScanExec(
options = relation.options,
hadoopConf =
relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options),
- metrics = allMetrics,
- alluxioPathReplacementMap)
+ metrics = allMetrics)
Some(reader)
} else {
None
@@ -645,8 +612,7 @@ case class GpuFileSourceScanExec(
optionalNumCoalescedBuckets,
QueryPlan.normalizePredicates(dataFilters, originalOutput),
None,
- queryUsesInputFile,
- alluxioPathsMap = alluxioPathsMap)(rapidsConf)
+ queryUsesInputFile)(rapidsConf)
}
}
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala
index d50b0b184ca..bfa17d1a8b4 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2022-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -47,8 +47,7 @@ class GpuReadAvroFileFormat extends AvroFileFormat with GpuReadFileFormatWithMet
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration,
- metrics: Map[String, GpuMetric],
- alluxioPathReplacementMap: Option[Map[String, String]] = None)
+ metrics: Map[String, GpuMetric])
: PartitionedFile => Iterator[InternalRow] = {
val sqlConf = sparkSession.sessionState.conf
val broadcastedHadoopConf =
diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala
deleted file mode 100644
index 4c39738e4d0..00000000000
--- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright (c) 2022-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": "320"}
-{"spark": "321"}
-{"spark": "321cdh"}
-{"spark": "322"}
-{"spark": "323"}
-{"spark": "324"}
-{"spark": "330"}
-{"spark": "330cdh"}
-{"spark": "330db"}
-{"spark": "331"}
-{"spark": "332"}
-{"spark": "332cdh"}
-{"spark": "333"}
-{"spark": "334"}
-spark-rapids-shim-json-lines ***/
-package com.nvidia.spark.rapids.shims
-
-import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex
-
-object FileIndexOptionsShims {
- val BASE_PATH_PARAM = PartitioningAwareFileIndex.BASE_PATH_PARAM
-}
diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala
index 4fc5c3efaf2..386b1209f76 100644
--- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala
+++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2022-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -45,7 +45,7 @@ import com.nvidia.spark.rapids._
import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression
import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex}
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
import org.apache.spark.sql.rapids.GpuFileSourceScanExec
class FileSourceScanExecMeta(plan: FileSourceScanExec,
@@ -95,46 +95,8 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec,
override def convertToGpu(): GpuExec = {
val sparkSession = wrapped.relation.sparkSession
val options = wrapped.relation.options
- val (location, alluxioPathsToReplaceMap) =
- if (AlluxioCfgUtils.enabledAlluxioReplacementAlgoConvertTime(conf)) {
- val shouldReadFromS3 = wrapped.relation.location match {
- // Only handle InMemoryFileIndex
- //
- // skip handle `MetadataLogFileIndex`, from the description of this class:
- // it's about the files generated by the `FileStreamSink`.
- // The streaming data source is not in our scope.
- //
- // For CatalogFileIndex and FileIndex of `delta` data source,
- // need more investigation.
- case inMemory: InMemoryFileIndex =>
- // List all the partitions to reduce overhead, pass in 2 empty filters.
- // Subsequent process will do the right partition pruning.
- // This operation is fast, because it lists files from the caches and the caches
- // already exist in the `InMemoryFileIndex`.
- val pds = inMemory.listFiles(Seq.empty, Seq.empty)
- AlluxioUtils.shouldReadDirectlyFromS3(conf, pds)
- case _ =>
- false
- }
-
- if (!shouldReadFromS3) {
- // it's convert time algorithm and some paths are not large tables
- AlluxioUtils.replacePathIfNeeded(
- conf,
- wrapped.relation,
- partitionFilters,
- wrapped.dataFilters)
- } else {
- // convert time algorithm and read large files
- (wrapped.relation.location, None)
- }
- } else {
- // it's not convert time algorithm or read large files, do not replace
- (wrapped.relation.location, None)
- }
-
val newRelation = HadoopFsRelation(
- location,
+ wrapped.relation.location,
wrapped.relation.partitionSchema,
wrapped.relation.dataSchema,
wrapped.relation.bucketSpec,
@@ -151,7 +113,6 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec,
wrapped.dataFilters,
wrapped.tableIdentifier,
wrapped.disableBucketedScan,
- queryUsesInputFile = false,
- alluxioPathsToReplaceMap)(conf)
+ queryUsesInputFile = false)(conf)
}
}
diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala
deleted file mode 100644
index 31b7461aea3..00000000000
--- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright (c) 2022-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": "320"}
-{"spark": "321"}
-{"spark": "321cdh"}
-{"spark": "322"}
-{"spark": "323"}
-{"spark": "324"}
-spark-rapids-shim-json-lines ***/
-package org.apache.spark.sql.types.shims
-
-import java.time.ZoneId
-
-import scala.util.Try
-
-import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName
-import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
-import org.apache.spark.sql.types.{AnyTimestampType, DataType, DateType}
-
-object PartitionValueCastShims {
- def isSupportedType(dt: DataType): Boolean = dt match {
- // Timestamp types
- case dt if AnyTimestampType.acceptsType(dt) => true
- case _ => false
- }
-
- // Only for TimestampType and TimestampNTZType
- def castTo(desiredType: DataType, value: String, zoneId: ZoneId): Any = desiredType match {
- // Copied from org/apache/spark/sql/execution/datasources/PartitionUtils.scala
- case dt if AnyTimestampType.acceptsType(desiredType) =>
- Try {
- Cast(Literal(unescapePathName(value)), dt, Some(zoneId.getId)).eval()
- }.getOrElse {
- Cast(Cast(Literal(value), DateType, Some(zoneId.getId)), dt).eval()
- }
- }
-}
diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala
deleted file mode 100644
index e1e1247b759..00000000000
--- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright (c) 2022-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": "330"}
-{"spark": "330cdh"}
-{"spark": "330db"}
-{"spark": "331"}
-{"spark": "332"}
-{"spark": "332cdh"}
-{"spark": "332db"}
-{"spark": "333"}
-{"spark": "334"}
-{"spark": "340"}
-{"spark": "341"}
-{"spark": "341db"}
-{"spark": "342"}
-{"spark": "343"}
-{"spark": "344"}
-{"spark": "350"}
-{"spark": "350db143"}
-{"spark": "351"}
-{"spark": "352"}
-{"spark": "353"}
-{"spark": "400"}
-spark-rapids-shim-json-lines ***/
-package org.apache.spark.sql.types.shims
-
-import java.time.ZoneId
-
-import scala.util.Try
-
-import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName
-import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
-import org.apache.spark.sql.types.{AnsiIntervalType, AnyTimestampType, DataType, DateType}
-
-object PartitionValueCastShims {
- def isSupportedType(dt: DataType): Boolean = dt match {
- case dt if AnyTimestampType.acceptsType(dt) => true
- case _: AnsiIntervalType => true
- case _ => false
- }
-
- // Only for AnsiIntervalType
- def castTo(desiredType: DataType, value: String, zoneId: ZoneId): Any = desiredType match {
- // Copied from org/apache/spark/sql/execution/datasources/PartitionUtils.scala
- case dt if AnyTimestampType.acceptsType(desiredType) =>
- Try {
- Cast(Literal(unescapePathName(value)), dt, Some(zoneId.getId)).eval()
- }.getOrElse {
- Cast(Cast(Literal(value), DateType, Some(zoneId.getId)), dt).eval()
- }
- case it: AnsiIntervalType =>
- Cast(Literal(unescapePathName(value)), it).eval()
- }
-}
diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala
index 2763af7fd56..64fdea89a34 100644
--- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala
+++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2022-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -27,7 +27,7 @@ import com.nvidia.spark.rapids._
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression}
import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex}
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
import org.apache.spark.sql.execution.datasources.json.JsonFileFormat
import org.apache.spark.sql.execution.exchange.ReusedExchangeExec
import org.apache.spark.sql.rapids.GpuFileSourceScanExec
@@ -134,46 +134,8 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec,
override def convertToGpu(): GpuExec = {
val sparkSession = wrapped.relation.sparkSession
val options = wrapped.relation.options
- val (location, alluxioPathsToReplaceMap) =
- if (AlluxioCfgUtils.enabledAlluxioReplacementAlgoConvertTime(conf)) {
- val shouldReadFromS3 = wrapped.relation.location match {
- // Only handle InMemoryFileIndex
- //
- // skip handle `MetadataLogFileIndex`, from the description of this class:
- // it's about the files generated by the `FileStreamSink`.
- // The streaming data source is not in our scope.
- //
- // For CatalogFileIndex and FileIndex of `delta` data source,
- // need more investigation.
- case inMemory: InMemoryFileIndex =>
- // List all the partitions to reduce overhead, pass in 2 empty filters.
- // Subsequent process will do the right partition pruning.
- // This operation is fast, because it lists files from the caches and the caches
- // already exist in the `InMemoryFileIndex`.
- val pds = inMemory.listFiles(Seq.empty, Seq.empty)
- AlluxioUtils.shouldReadDirectlyFromS3(conf, pds)
- case _ =>
- false
- }
-
- if (!shouldReadFromS3) {
- // it's convert time algorithm and some paths are not large tables
- AlluxioUtils.replacePathIfNeeded(
- conf,
- wrapped.relation,
- partitionFilters,
- dataFilters)
- } else {
- // convert time algorithm and read large files
- (wrapped.relation.location, None)
- }
- } else {
- // it's not convert time algorithm or read large files, do not replace
- (wrapped.relation.location, None)
- }
-
val newRelation = HadoopFsRelation(
- location,
+ wrapped.relation.location,
wrapped.relation.partitionSchema,
wrapped.relation.dataSchema,
wrapped.relation.bucketSpec,
@@ -191,7 +153,6 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec,
dataFilters,
wrapped.tableIdentifier,
wrapped.disableBucketedScan,
- queryUsesInputFile = false,
- alluxioPathsToReplaceMap)(conf)
+ queryUsesInputFile = false)(conf)
}
}
diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala
deleted file mode 100644
index 5cd4cafe7f0..00000000000
--- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright (c) 2022-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": "332db"}
-{"spark": "340"}
-{"spark": "341"}
-{"spark": "341db"}
-{"spark": "342"}
-{"spark": "343"}
-{"spark": "344"}
-{"spark": "350"}
-{"spark": "350db143"}
-{"spark": "351"}
-{"spark": "352"}
-{"spark": "353"}
-{"spark": "400"}
-spark-rapids-shim-json-lines ***/
-package com.nvidia.spark.rapids.shims
-
-import org.apache.spark.sql.execution.datasources.FileIndexOptions
-
-object FileIndexOptionsShims {
- val BASE_PATH_PARAM = FileIndexOptions.BASE_PATH_PARAM
-}
diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioConfigReaderSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioConfigReaderSuite.scala
deleted file mode 100644
index ec8db0bafc2..00000000000
--- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioConfigReaderSuite.scala
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * Copyright (c) 2022-2023, 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.
- */
-package com.nvidia.spark.rapids
-
-import java.io.File
-import java.nio.charset.StandardCharsets
-import java.nio.file.Files
-
-import org.scalatest.funsuite.AnyFunSuite
-
-class AlluxioConfigReaderSuite extends AnyFunSuite {
-
- test("testReadAlluxioMasterAndPort") {
- val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix")
- val confDir = new File(homeDir.toFile, "conf")
- assert(confDir.mkdir())
- val f = new File(confDir, "alluxio-site.properties")
-
- try {
- val content =
- """
-alluxio.master.hostname=host1.com
-alluxio.master.rpc.port=200
- """
- Files.write(f.toPath, content.getBytes(StandardCharsets.UTF_8))
- val (host, port) = new AlluxioConfigReader()
- .readMasterAndPort(homeDir.toFile.getAbsolutePath)
- assert(host.equals("host1.com"))
- assert(port == "200")
- } finally {
- f.delete()
- confDir.delete()
- homeDir.toFile.delete()
- }
- }
-
- test("testReadAlluxioMasterAndPort, get default port") {
- val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix")
- val confDir = new File(homeDir.toFile, "conf")
- assert(confDir.mkdir())
- val f = new File(confDir, "alluxio-site.properties")
-
- try {
- val content =
- """
-alluxio.master.hostname=host1.com
- """
- Files.write(f.toPath, content.getBytes(StandardCharsets.UTF_8))
- val (host, port) = new AlluxioConfigReader()
- .readMasterAndPort(homeDir.toFile.getAbsolutePath)
- assert(host.equals("host1.com"))
- assert(port == "19998")
- } finally {
- f.delete()
- confDir.delete()
- homeDir.toFile.delete()
- }
- }
-
- test("testReadAlluxioMasterAndPort, cfg does not specify master") {
- val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix")
- val confDir = new File(homeDir.toFile, "conf")
- assert(confDir.mkdir())
- val f = new File(confDir, "alluxio-site.properties")
-
- try {
- val content =
- """
-xxx=yyy
- """
- Files.write(f.toPath, content.getBytes(StandardCharsets.UTF_8))
- try {
- new AlluxioConfigReader()
- .readMasterAndPort(homeDir.toFile.getAbsolutePath)
- assert(false)
- } catch {
- case e: RuntimeException =>
- assert(e.getMessage.contains("Can't find alluxio.master.hostname"))
- }
- } finally {
- f.delete()
- confDir.delete()
- homeDir.toFile.delete()
- }
- }
-
- test("testReadAlluxioMasterAndPort, cfg file does not exist") {
- val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix")
- val confDir = new File(homeDir.toFile, "conf")
- assert(confDir.mkdir())
-
- try {
- try {
- new AlluxioConfigReader()
- .readMasterAndPort(homeDir.toFile.getAbsolutePath)
- assert(false)
- } catch {
- case e: RuntimeException =>
- assert(e.getMessage.contains("Alluxio config file not found in"))
- }
- } finally {
- confDir.delete()
- homeDir.toFile.delete()
- }
- }
-}
diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioUtilsSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioUtilsSuite.scala
deleted file mode 100644
index 609778e8a0a..00000000000
--- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioUtilsSuite.scala
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Copyright (c) 2022-2023, 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.
- */
-package com.nvidia.spark.rapids
-
-import scala.collection.mutable
-
-import com.nvidia.spark.rapids.shims.PartitionedFileUtilsShim
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileStatus, Path}
-import org.mockito.Mockito._
-import org.scalatest.funsuite.AnyFunSuite
-import org.scalatestplus.mockito.MockitoSugar.mock
-
-import org.apache.spark.sql.RuntimeConfig
-import org.apache.spark.sql.execution.datasources.{PartitionDirectory, PartitionedFile}
-import org.apache.spark.sql.internal.SQLConf
-
-class AlluxioMasterAndPortReaderMock(master: String, port: String)
- extends AlluxioConfigReader {
- override def readAlluxioMasterAndPort(conf: RapidsConf): (String, String) = (master, port)
-}
-
-class AlluxioFSMock extends AlluxioFS {
- private val mountPoints = mutable.Map[String, String]()
-
- /**
- * Get S3 mount points by Alluxio client
- *
- * @return mount points map, key of map is Alluxio path, value of map is S3 path.
- * E.g.: returns a map: {'/bucket_1': 's3://bucket_1'}
- */
- override def getExistingMountPoints(): mutable.Map[String, String] = {
- mountPoints
- }
-
- /**
- * Mount an S3 path to Alluxio
- *
- * @param alluxioPath Alluxio path
- * @param s3Path S3 path
- */
- override def mount(alluxioPath: String, s3Path: String): Unit = {
- mountPoints(alluxioPath) = s3Path
- }
-
- def getMountPoints(): mutable.Map[String, String] = {
- mountPoints
- }
-}
-
-class AlluxioUtilsSuite extends AnyFunSuite {
-
- def setMockOnAlluxioUtils(): Unit = {
- AlluxioUtils.setAlluxioFS(new AlluxioFSMock())
- AlluxioUtils.setAlluxioMasterAndPortReader(
- new AlluxioMasterAndPortReaderMock("localhost", "19998"))
- }
-
- test("updateFilesTaskTimeIfAlluxio") {
- setMockOnAlluxioUtils()
- AlluxioUtils.resetInitInfo()
- val replaceMap = Map[String, String](("s3a://", "alluxio://localhost:19998/"))
- val partitionedFiles = Array[PartitionedFile](
- PartitionedFileUtilsShim.newPartitionedFile(null, "s3a://bucket_1/a.file", 0, 0),
- PartitionedFileUtilsShim.newPartitionedFile(null, "s3a://bucket_2/b.file", 0, 0),
- PartitionedFileUtilsShim.newPartitionedFile(null, "myScheme://bucket_1/1.file", 0, 0)
- )
- val replaced = AlluxioUtils.updateFilesTaskTimeIfAlluxio(partitionedFiles, Option(replaceMap))
- assert(replaced.size == 3)
- assert(replaced(0).toRead.filePath.toString === "alluxio://localhost:19998/bucket_1/a.file")
- assert(replaced(0).original.get.filePath.toString === "s3a://bucket_1/a.file")
- assert(replaced(1).toRead.filePath.toString === "alluxio://localhost:19998/bucket_2/b.file")
- assert(replaced(1).original.get.filePath.toString === "s3a://bucket_2/b.file")
- assert(replaced(2).toRead.filePath.toString === "myScheme://bucket_1/1.file")
- assert(replaced(2).original.isEmpty)
- }
-
- test("updateFilesTaskTimeIfAlluxio, multiple replacing rules") {
- setMockOnAlluxioUtils()
- AlluxioUtils.resetInitInfo()
- val replaceMap = Map[String, String](
- ("s3a://", "alluxio://localhost:19998/"), // the first rule
- ("s3a://bucket_1", "alluxio://localhost:19998/") // should not specify this rule!
- )
- val partitionedFiles = Array[PartitionedFile](
- PartitionedFileUtilsShim.newPartitionedFile(null, "s3a://bucket_1/a.file", 0, 0)
- )
- try {
- AlluxioUtils.updateFilesTaskTimeIfAlluxio(partitionedFiles, Option(replaceMap))
- assert(false)
- } catch {
- case e: IllegalArgumentException =>
- assert(e.getMessage.contains("same replacing rules"))
- }
- }
-
- test("checkIfNeedsReplaced for PathsToReplace map, true") {
- setMockOnAlluxioUtils()
- AlluxioUtils.resetInitInfo()
- val sqlConf = new SQLConf()
- sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace",
- "s3a://bucket_1->alluxio://0.1.2.3:19998/foo")
- val rapidsConf = new RapidsConf(sqlConf)
- val fs = new FileStatus(0, false, 1, 1024L, 0L,
- new Path("s3a://bucket_1/a.parquet"))
- val pds = Seq(PartitionDirectory(null, Array(fs)))
- val configuration = new Configuration()
- val runtimeConfig = mock[RuntimeConfig]
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key"))
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key"))
-
- assert(AlluxioUtils
- .checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isDefined)
- }
-
- test("checkIfNeedsReplaced for PathsToReplace map, false") {
- setMockOnAlluxioUtils()
- AlluxioUtils.resetInitInfo()
- val sqlConf = new SQLConf()
- sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace",
- "s3a://bucket_1->alluxio://0.1.2.3:19998/foo")
- val rapidsConf = new RapidsConf(sqlConf)
- val fs = new FileStatus(0, false, 1, 1024L, 0L,
- new Path("s3a://not_found/a.parquet"))
- val pds = Seq(PartitionDirectory(null, Array(fs)))
- val configuration = new Configuration()
- val runtimeConfig = mock[RuntimeConfig]
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key"))
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key"))
-
- assert(AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isEmpty)
- }
-
- test("checkIfNeedsReplaced for PathsToReplace map, exception") {
- setMockOnAlluxioUtils()
- AlluxioUtils.resetInitInfo()
- val sqlConf = new SQLConf()
- sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace",
- "s3a://bucket_1->alluxio://0.1.2.3:19998/dir1," +
- "s3a://bucket_1/dir1->alluxio://4.4.4.4:19998/dir1"
- )
- val rapidsConf = new RapidsConf(sqlConf)
- val fs = new FileStatus(0, false, 1, 1024L, 0L,
- new Path("s3a://bucket_1/dir1/a.parquet")) // matches 2 rules
- val pds = Seq(PartitionDirectory(null, Array(fs)))
- val configuration = new Configuration()
- val runtimeConfig = mock[RuntimeConfig]
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key"))
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key"))
-
- try {
- AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isEmpty
- assert(false)
- } catch {
- case e: IllegalArgumentException =>
- assert(e.getMessage.contains("same replacing rules"))
- }
- }
-
- test("checkIfNeedsReplaced for PathsToReplace map, invalid setting") {
- setMockOnAlluxioUtils()
- AlluxioUtils.resetInitInfo()
- val sqlConf = new SQLConf()
- sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace",
- "s3a://bucket_1->alluxio://0.1.2.3:19998/->dir1" // contains 2 `->`
- )
- val rapidsConf = new RapidsConf(sqlConf)
- val fs = new FileStatus(0, false, 1, 1024L, 0L,
- new Path("s3a://bucket_1/dir1/a.parquet")) // matches 2 rules
- val pds = Seq(PartitionDirectory(null, Array(fs)))
- val configuration = new Configuration()
- val runtimeConfig = mock[RuntimeConfig]
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key"))
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key"))
-
- try {
- AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isEmpty
- assert(false)
- } catch {
- case e: IllegalArgumentException =>
- assert(e.getMessage.contains("Invalid setting"))
- }
- }
-
- test("autoMountIfNeeded, auto-mount is false") {
- setMockOnAlluxioUtils()
- AlluxioUtils.resetInitInfo()
- val sqlConf = new SQLConf()
- sqlConf.setConfString("spark.rapids.alluxio.automount.enabled", "false")
- val rapidsConf = new RapidsConf(sqlConf)
- val fs = new FileStatus(0, false, 1, 1024L, 0L,
- new Path("s3a://bucket_1/a.parquet"))
- val pds = Seq(PartitionDirectory(null, Array(fs)))
- val configuration = new Configuration()
- val runtimeConfig = mock[RuntimeConfig]
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key"))
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key"))
-
- assert(AlluxioUtils.autoMountIfNeeded(rapidsConf, pds, configuration, runtimeConfig).isEmpty)
- }
-
- test("autoMountIfNeeded, auto-mount is true") {
- setMockOnAlluxioUtils()
- val alluxioFSMock = new AlluxioFSMock()
- AlluxioUtils.setAlluxioFS(alluxioFSMock)
- AlluxioUtils.resetInitInfo()
- val sqlConf = new SQLConf()
- sqlConf.setConfString("spark.rapids.alluxio.automount.enabled", "true")
- val rapidsConf = new RapidsConf(sqlConf)
-
- val configuration = new Configuration()
- val runtimeConfig = mock[RuntimeConfig]
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key"))
- when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key"))
-
- assert(alluxioFSMock.getMountPoints().isEmpty)
- val fs = new FileStatus(0, false, 1, 1024L, 0L,
- new Path("s3a://bucket_1/a.parquet"))
- val pds = Seq(PartitionDirectory(null, Array(fs)))
- assert(AlluxioUtils.autoMountIfNeeded(rapidsConf, pds, configuration, runtimeConfig).isDefined)
- assert(alluxioFSMock.getMountPoints().contains("/bucket_1"))
-
- val fs2 = new FileStatus(0, false, 1, 1024L, 0L,
- new Path("myScheme://bucket_2/a.parquet"))
- val pds2 = Seq(PartitionDirectory(null, Array(fs2)))
- assert(AlluxioUtils.autoMountIfNeeded(rapidsConf, pds2, configuration, runtimeConfig).isEmpty)
- assert(alluxioFSMock.getMountPoints().size == 1)
- }
-}
diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala
index 21f364e085b..ae1b5a47011 100644
--- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala
+++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2021-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2021-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -58,7 +58,6 @@ class GpuMultiFileReaderSuite extends AnyFunSuite {
override def getBatchRunner(
tc: TaskContext,
file: PartitionedFile,
- origFile: Option[PartitionedFile],
conf: Configuration,
filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = {
() => null
diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala
index e303c207f63..a2dcaa09e45 100644
--- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala
+++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -90,11 +90,11 @@ class DynamicPruningSuite
case BroadcastQueryStageExec(id, plan, _canonicalized) =>
val newPlan = replaceSubquery(plan)
BroadcastQueryStageExec(id, newPlan, _canonicalized)
- case g @ GpuFileSourceScanExec(r, o, rs, pf, obs, oncb, df, ti, dbs, quif, apm, rps) =>
+ case g @ GpuFileSourceScanExec(r, o, rs, pf, obs, oncb, df, ti, dbs, quif, rps) =>
val newPartitionFilters = updatePartitionFilters(pf)
val rc = g.rapidsConf
GpuFileSourceScanExec(r, o, rs, newPartitionFilters,
- obs, oncb, df, ti, dbs, quif, apm, rps)(rc)
+ obs, oncb, df, ti, dbs, quif, rps)(rc)
case FileSourceScanExec(r, o, rs, pf, obs, oncb, df, ti, dbs) =>
val newPartitionFilters = updatePartitionFilters(pf)
FileSourceScanExec(r, o, rs, newPartitionFilters, obs, oncb, df, ti, dbs)