diff --git a/aggregator/pom.xml b/aggregator/pom.xml index 9ba0a1088e5..d9a8380c643 100644 --- a/aggregator/pom.xml +++ b/aggregator/pom.xml @@ -711,6 +711,23 @@ + + release350db + + + buildver + 350db + + + + + com.nvidia + rapids-4-spark-delta-spark350db_${scala.binary.version} + ${project.version} + ${spark.version.classifier} + + + release351 diff --git a/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala b/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala index 9134505c2f2..03b6320024a 100644 --- a/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala +++ b/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} spark-rapids-shim-json-lines ***/ diff --git a/delta-lake/README.md b/delta-lake/README.md index fad78dca706..af8c007eeb8 100644 --- a/delta-lake/README.md +++ b/delta-lake/README.md @@ -19,6 +19,7 @@ and directory contains the corresponding support code. | Databricks 11.3 | Databricks 11.3 | `delta-spark330db` | | Databricks 12.2 | Databricks 12.2 | `delta-spark332db` | | Databricks 13.3 | Databricks 13.3 | `delta-spark341db` | +| Databricks 14.3 | Databricks 14.3 | `delta-spark350db` | Delta Lake is not supported on all Spark versions, and for Spark versions where it is not supported the `delta-stub` project is used. diff --git a/delta-lake/common/src/main/databricks/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaLog.scala b/delta-lake/common/src/main/databricks/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaLog.scala index 2927b8607ad..a291b8c5a75 100644 --- a/delta-lake/common/src/main/databricks/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaLog.scala +++ b/delta-lake/common/src/main/databricks/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaLog.scala @@ -68,7 +68,7 @@ object GpuDeltaLog { dataPath: String, options: Map[String, String], rapidsConf: RapidsConf): GpuDeltaLog = { - val deltaLog = DeltaLog.forTable(spark, dataPath, options) + val deltaLog = DeltaLog.forTable(spark, new Path(dataPath), options) new GpuDeltaLog(deltaLog, rapidsConf) } diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DatabricksDeltaProviderBase.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DatabricksDeltaProviderBase.scala index 55f9cc2ae49..f35b723ac06 100644 --- a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DatabricksDeltaProviderBase.scala +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DatabricksDeltaProviderBase.scala @@ -337,7 +337,7 @@ class DeltaCreatableRelationProviderMeta( } val path = saveCmd.options.get("path") if (path.isDefined) { - val deltaLog = DeltaLog.forTable(SparkSession.active, path.get, saveCmd.options) + val deltaLog = DeltaLog.forTable(SparkSession.active, new Path(path.get), saveCmd.options) RapidsDeltaUtils.tagForDeltaWrite(this, saveCmd.query.schema, Some(deltaLog), saveCmd.options, SparkSession.active) } else { @@ -346,4 +346,4 @@ class DeltaCreatableRelationProviderMeta( } override def convertToGpu(): GpuCreatableRelationProvider = new GpuDeltaDataSource(conf) -} \ No newline at end of file +} diff --git a/delta-lake/common/src/main/databricks/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala similarity index 100% rename from delta-lake/common/src/main/databricks/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala rename to delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala new file mode 100644 index 00000000000..b6e9e11946d --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala @@ -0,0 +1,193 @@ +/* + * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * + * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.actions.FileAction +import com.databricks.sql.transaction.tahoe.constraints.{Constraint, DeltaInvariantCheckerExec} +import com.databricks.sql.transaction.tahoe.files.TahoeBatchFileIndex +import com.databricks.sql.transaction.tahoe.metering.DeltaLogging +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.rapids.GpuShuffleEnv +import org.apache.spark.sql.rapids.GpuV1WriteUtils.GpuEmpty2Null +import org.apache.spark.sql.rapids.delta.{DeltaShufflePartitionsUtil, GpuOptimizeWriteExchangeExec, OptimizeWriteExchangeExec} +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.Clock + +/** + * Used to perform a set of reads in a transaction and then commit a set of updates to the + * state of the log. All reads from the DeltaLog, MUST go through this instance rather + * than directly to the DeltaLog otherwise they will not be check for logical conflicts + * with concurrent updates. + * + * This class is not thread-safe. + * + * @param deltaLog The Delta Log for the table this transaction is modifying. + * @param snapshot The snapshot that this transaction is reading at. + * @param rapidsConf RAPIDS Accelerator config settings. + */ +abstract class GpuOptimisticTransactionBase + (deltaLog: DeltaLog, snapshot: Snapshot, val rapidsConf: RapidsConf) + (implicit clock: Clock) + extends OptimisticTransaction(deltaLog, snapshot)(clock) + with DeltaLogging { + + /** + * Adds checking of constraints on the table + * @param plan Plan to generate the table to check against constraints + * @param constraints Constraints to check on the table + * @return GPU columnar plan to execute + */ + protected def addInvariantChecks(plan: SparkPlan, constraints: Seq[Constraint]): SparkPlan = { + val cpuInvariants = + DeltaInvariantCheckerExec.buildInvariantChecks(plan.output, constraints, plan.session) + GpuCheckDeltaInvariant.maybeConvertToGpu(cpuInvariants, rapidsConf) match { + case Some(gpuInvariants) => + val gpuPlan = convertToGpu(plan) + GpuDeltaInvariantCheckerExec(gpuPlan, gpuInvariants) + case None => + val cpuPlan = convertToCpu(plan) + DeltaInvariantCheckerExec(cpuPlan, constraints) + } + } + + /** GPU version of convertEmptyToNullIfNeeded */ + private def gpuConvertEmptyToNullIfNeeded( + plan: GpuExec, + partCols: Seq[Attribute], + constraints: Seq[Constraint]): SparkPlan = { + if (!spark.conf.get(DeltaSQLConf.CONVERT_EMPTY_TO_NULL_FOR_STRING_PARTITION_COL)) { + return plan + } + // No need to convert if there are no constraints. The empty strings will be converted later by + // FileFormatWriter and FileFormatDataWriter. Note that we might still do unnecessary convert + // here as the constraints might not be related to the string partition columns. A precise + // check will need to walk the constraints to see if such columns are really involved. It + // doesn't seem to worth the effort. + if (constraints.isEmpty) return plan + + val partSet = AttributeSet(partCols) + var needConvert = false + val projectList: Seq[NamedExpression] = plan.output.map { + case p if partSet.contains(p) && p.dataType == StringType => + needConvert = true + GpuAlias(GpuEmpty2Null(p), p.name)() + case attr => attr + } + if (needConvert) GpuProjectExec(projectList.toList, plan) else plan + } + + /** + * If there is any string partition column and there are constraints defined, add a projection to + * convert empty string to null for that column. The empty strings will be converted to null + * eventually even without this convert, but we want to do this earlier before check constraints + * so that empty strings are correctly rejected. Note that this should not cause the downstream + * logic in `FileFormatWriter` to add duplicate conversions because the logic there checks the + * partition column using the original plan's output. When the plan is modified with additional + * projections, the partition column check won't match and will not add more conversion. + * + * @param plan The original SparkPlan. + * @param partCols The partition columns. + * @param constraints The defined constraints. + * @return A SparkPlan potentially modified with an additional projection on top of `plan` + */ + override def convertEmptyToNullIfNeeded( + plan: SparkPlan, + partCols: Seq[Attribute], + constraints: Seq[Constraint]): SparkPlan = { + // Reuse the CPU implementation if the plan ends up on the CPU, otherwise do the + // equivalent on the GPU. + plan match { + case g: GpuExec => gpuConvertEmptyToNullIfNeeded(g, partCols, constraints) + case _ => super.convertEmptyToNullIfNeeded(plan, partCols, constraints) + } + } + + override def writeFiles( + inputData: Dataset[_], + additionalConstraints: Seq[Constraint]): Seq[FileAction] = { + writeFiles(inputData, None, additionalConstraints) + } + + protected def applyOptimizeWriteIfNeeded( + spark: SparkSession, + physicalPlan: SparkPlan, + partitionSchema: StructType, + isOptimize: Boolean): SparkPlan = { + val optimizeWriteEnabled = !isOptimize && + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_WRITE_ENABLED) + .orElse(DeltaConfigs.OPTIMIZE_WRITE.fromMetaData(metadata)).getOrElse(false) + if (optimizeWriteEnabled) { + val planWithoutTopRepartition = + DeltaShufflePartitionsUtil.removeTopRepartition(physicalPlan) + val partitioning = DeltaShufflePartitionsUtil.partitioningForRebalance( + physicalPlan.output, partitionSchema, spark.sessionState.conf.numShufflePartitions) + planWithoutTopRepartition match { + case p: GpuExec => + val partMeta = GpuOverrides.wrapPart(partitioning, rapidsConf, None) + partMeta.tagForGpu() + if (partMeta.canThisBeReplaced) { + val plan = GpuOptimizeWriteExchangeExec(partMeta.convertToGpu(), p) + if (GpuShuffleEnv.useGPUShuffle(rapidsConf)) { + GpuCoalesceBatches(plan, TargetSize(rapidsConf.gpuTargetBatchSizeBytes)) + } else { + GpuShuffleCoalesceExec(plan, rapidsConf.gpuTargetBatchSizeBytes) + } + } else { + GpuColumnarToRowExec(OptimizeWriteExchangeExec(partitioning, p)) + } + case p => + OptimizeWriteExchangeExec(partitioning, p) + } + } else { + physicalPlan + } + } + + protected def isOptimizeCommand(plan: LogicalPlan): Boolean = { + val leaves = plan.collectLeaves() + leaves.size == 1 && leaves.head.collect { + case LogicalRelation(HadoopFsRelation( + index: TahoeBatchFileIndex, _, _, _, _, _), _, _, _) => + index.actionType.equals("Optimize") + }.headOption.getOrElse(false) + } + + protected def convertToCpu(plan: SparkPlan): SparkPlan = plan match { + case GpuRowToColumnarExec(p, _) => p + case p: GpuExec => GpuColumnarToRowExec(p) + case p => p + } + + protected def convertToGpu(plan: SparkPlan): SparkPlan = plan match { + case GpuColumnarToRowExec(p, _) => p + case p: GpuExec => p + case p => GpuRowToColumnarExec(p, TargetSize(rapidsConf.gpuTargetBatchSizeBytes)) + } +} diff --git a/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala new file mode 100644 index 00000000000..b6e9e11946d --- /dev/null +++ b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala @@ -0,0 +1,193 @@ +/* + * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * + * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.actions.FileAction +import com.databricks.sql.transaction.tahoe.constraints.{Constraint, DeltaInvariantCheckerExec} +import com.databricks.sql.transaction.tahoe.files.TahoeBatchFileIndex +import com.databricks.sql.transaction.tahoe.metering.DeltaLogging +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.rapids.GpuShuffleEnv +import org.apache.spark.sql.rapids.GpuV1WriteUtils.GpuEmpty2Null +import org.apache.spark.sql.rapids.delta.{DeltaShufflePartitionsUtil, GpuOptimizeWriteExchangeExec, OptimizeWriteExchangeExec} +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.Clock + +/** + * Used to perform a set of reads in a transaction and then commit a set of updates to the + * state of the log. All reads from the DeltaLog, MUST go through this instance rather + * than directly to the DeltaLog otherwise they will not be check for logical conflicts + * with concurrent updates. + * + * This class is not thread-safe. + * + * @param deltaLog The Delta Log for the table this transaction is modifying. + * @param snapshot The snapshot that this transaction is reading at. + * @param rapidsConf RAPIDS Accelerator config settings. + */ +abstract class GpuOptimisticTransactionBase + (deltaLog: DeltaLog, snapshot: Snapshot, val rapidsConf: RapidsConf) + (implicit clock: Clock) + extends OptimisticTransaction(deltaLog, snapshot)(clock) + with DeltaLogging { + + /** + * Adds checking of constraints on the table + * @param plan Plan to generate the table to check against constraints + * @param constraints Constraints to check on the table + * @return GPU columnar plan to execute + */ + protected def addInvariantChecks(plan: SparkPlan, constraints: Seq[Constraint]): SparkPlan = { + val cpuInvariants = + DeltaInvariantCheckerExec.buildInvariantChecks(plan.output, constraints, plan.session) + GpuCheckDeltaInvariant.maybeConvertToGpu(cpuInvariants, rapidsConf) match { + case Some(gpuInvariants) => + val gpuPlan = convertToGpu(plan) + GpuDeltaInvariantCheckerExec(gpuPlan, gpuInvariants) + case None => + val cpuPlan = convertToCpu(plan) + DeltaInvariantCheckerExec(cpuPlan, constraints) + } + } + + /** GPU version of convertEmptyToNullIfNeeded */ + private def gpuConvertEmptyToNullIfNeeded( + plan: GpuExec, + partCols: Seq[Attribute], + constraints: Seq[Constraint]): SparkPlan = { + if (!spark.conf.get(DeltaSQLConf.CONVERT_EMPTY_TO_NULL_FOR_STRING_PARTITION_COL)) { + return plan + } + // No need to convert if there are no constraints. The empty strings will be converted later by + // FileFormatWriter and FileFormatDataWriter. Note that we might still do unnecessary convert + // here as the constraints might not be related to the string partition columns. A precise + // check will need to walk the constraints to see if such columns are really involved. It + // doesn't seem to worth the effort. + if (constraints.isEmpty) return plan + + val partSet = AttributeSet(partCols) + var needConvert = false + val projectList: Seq[NamedExpression] = plan.output.map { + case p if partSet.contains(p) && p.dataType == StringType => + needConvert = true + GpuAlias(GpuEmpty2Null(p), p.name)() + case attr => attr + } + if (needConvert) GpuProjectExec(projectList.toList, plan) else plan + } + + /** + * If there is any string partition column and there are constraints defined, add a projection to + * convert empty string to null for that column. The empty strings will be converted to null + * eventually even without this convert, but we want to do this earlier before check constraints + * so that empty strings are correctly rejected. Note that this should not cause the downstream + * logic in `FileFormatWriter` to add duplicate conversions because the logic there checks the + * partition column using the original plan's output. When the plan is modified with additional + * projections, the partition column check won't match and will not add more conversion. + * + * @param plan The original SparkPlan. + * @param partCols The partition columns. + * @param constraints The defined constraints. + * @return A SparkPlan potentially modified with an additional projection on top of `plan` + */ + override def convertEmptyToNullIfNeeded( + plan: SparkPlan, + partCols: Seq[Attribute], + constraints: Seq[Constraint]): SparkPlan = { + // Reuse the CPU implementation if the plan ends up on the CPU, otherwise do the + // equivalent on the GPU. + plan match { + case g: GpuExec => gpuConvertEmptyToNullIfNeeded(g, partCols, constraints) + case _ => super.convertEmptyToNullIfNeeded(plan, partCols, constraints) + } + } + + override def writeFiles( + inputData: Dataset[_], + additionalConstraints: Seq[Constraint]): Seq[FileAction] = { + writeFiles(inputData, None, additionalConstraints) + } + + protected def applyOptimizeWriteIfNeeded( + spark: SparkSession, + physicalPlan: SparkPlan, + partitionSchema: StructType, + isOptimize: Boolean): SparkPlan = { + val optimizeWriteEnabled = !isOptimize && + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_WRITE_ENABLED) + .orElse(DeltaConfigs.OPTIMIZE_WRITE.fromMetaData(metadata)).getOrElse(false) + if (optimizeWriteEnabled) { + val planWithoutTopRepartition = + DeltaShufflePartitionsUtil.removeTopRepartition(physicalPlan) + val partitioning = DeltaShufflePartitionsUtil.partitioningForRebalance( + physicalPlan.output, partitionSchema, spark.sessionState.conf.numShufflePartitions) + planWithoutTopRepartition match { + case p: GpuExec => + val partMeta = GpuOverrides.wrapPart(partitioning, rapidsConf, None) + partMeta.tagForGpu() + if (partMeta.canThisBeReplaced) { + val plan = GpuOptimizeWriteExchangeExec(partMeta.convertToGpu(), p) + if (GpuShuffleEnv.useGPUShuffle(rapidsConf)) { + GpuCoalesceBatches(plan, TargetSize(rapidsConf.gpuTargetBatchSizeBytes)) + } else { + GpuShuffleCoalesceExec(plan, rapidsConf.gpuTargetBatchSizeBytes) + } + } else { + GpuColumnarToRowExec(OptimizeWriteExchangeExec(partitioning, p)) + } + case p => + OptimizeWriteExchangeExec(partitioning, p) + } + } else { + physicalPlan + } + } + + protected def isOptimizeCommand(plan: LogicalPlan): Boolean = { + val leaves = plan.collectLeaves() + leaves.size == 1 && leaves.head.collect { + case LogicalRelation(HadoopFsRelation( + index: TahoeBatchFileIndex, _, _, _, _, _), _, _, _) => + index.actionType.equals("Optimize") + }.headOption.getOrElse(false) + } + + protected def convertToCpu(plan: SparkPlan): SparkPlan = plan match { + case GpuRowToColumnarExec(p, _) => p + case p: GpuExec => GpuColumnarToRowExec(p) + case p => p + } + + protected def convertToGpu(plan: SparkPlan): SparkPlan = plan match { + case GpuColumnarToRowExec(p, _) => p + case p: GpuExec => p + case p => GpuRowToColumnarExec(p, TargetSize(rapidsConf.gpuTargetBatchSizeBytes)) + } +} 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 e109b81f1e5..088a2a788da 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 @@ -113,7 +113,7 @@ object GpuDeltaParquetFileFormat { meta.willNotWorkOnGpu( s"reading metadata column $IS_ROW_DELETED_COLUMN_NAME is not supported") } - if (format.hasDeletionVectorMap()) { + if (format.hasDeletionVectorMap) { meta.willNotWorkOnGpu("deletion vectors are not supported") } } diff --git a/delta-lake/delta-spark350db/pom.xml b/delta-lake/delta-spark350db/pom.xml new file mode 100644 index 00000000000..122ad171a26 --- /dev/null +++ b/delta-lake/delta-spark350db/pom.xml @@ -0,0 +1,85 @@ + + + + 4.0.0 + + + com.nvidia + rapids-4-spark-shim-deps-parent_2.12 + 24.10.0-SNAPSHOT + ../../shim-deps/pom.xml + + + rapids-4-spark-delta-spark350db_2.12 + RAPIDS Accelerator for Apache Spark Databricks 13.3 Delta Lake Support + Databricks 13.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark + 24.10.0-SNAPSHOT + + + false + **/* + package + + + + + org.roaringbitmap + RoaringBitmap + + + com.nvidia + rapids-4-spark-sql_${scala.binary.version} + ${project.version} + ${spark.version.classifier} + provided + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-common-sources + generate-sources + + add-source + + + + ${project.basedir}/../common/src/main/scala + ${project.basedir}/../common/src/main/databricks/scala + + + + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.rat + apache-rat-plugin + + + + diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuCreateDeltaTableCommand.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuCreateDeltaTableCommand.scala new file mode 100644 index 00000000000..320485eb1ee --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuCreateDeltaTableCommand.scala @@ -0,0 +1,464 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from CreateDeltaTableCommand.scala in the + * Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.actions.Metadata +import com.databricks.sql.transaction.tahoe.commands.{TableCreationModes, WriteIntoDelta} +import com.databricks.sql.transaction.tahoe.metering.DeltaLogging +import com.databricks.sql.transaction.tahoe.schema.SchemaUtils +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.nvidia.spark.rapids.RapidsConf +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.execution.command.{LeafRunnableCommand, RunnableCommand} +import org.apache.spark.sql.types.StructType + +/** + * Single entry point for all write or declaration operations for Delta tables accessed through + * the table name. + * + * @param table The table identifier for the Delta table + * @param existingTableOpt The existing table for the same identifier if exists + * @param mode The save mode when writing data. Relevant when the query is empty or set to Ignore + * with `CREATE TABLE IF NOT EXISTS`. + * @param query The query to commit into the Delta table if it exist. This can come from + * - CTAS + * - saveAsTable + */ +case class GpuCreateDeltaTableCommand( + table: CatalogTable, + existingTableOpt: Option[CatalogTable], + mode: SaveMode, + query: Option[LogicalPlan], + operation: TableCreationModes.CreationMode = TableCreationModes.Create, + tableByPath: Boolean = false, + override val output: Seq[Attribute] = Nil)(@transient rapidsConf: RapidsConf) + extends LeafRunnableCommand + with DeltaLogging { + + override def otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) + + override def run(sparkSession: SparkSession): Seq[Row] = { + val table = this.table + + assert(table.tableType != CatalogTableType.VIEW) + assert(table.identifier.database.isDefined, "Database should've been fixed at analysis") + // There is a subtle race condition here, where the table can be created by someone else + // while this command is running. Nothing we can do about that though :( + val tableExists = existingTableOpt.isDefined + if (mode == SaveMode.Ignore && tableExists) { + // Early exit on ignore + return Nil + } else if (mode == SaveMode.ErrorIfExists && tableExists) { + throw DeltaErrors.tableAlreadyExists(table) + } + + val tableWithLocation = if (tableExists) { + val existingTable = existingTableOpt.get + table.storage.locationUri match { + case Some(location) if location.getPath != existingTable.location.getPath => + throw DeltaErrors.tableLocationMismatch(table, existingTable) + case _ => + } + table.copy( + storage = existingTable.storage, + tableType = existingTable.tableType) + } else if (table.storage.locationUri.isEmpty) { + // We are defining a new managed table + assert(table.tableType == CatalogTableType.MANAGED) + val loc = sparkSession.sessionState.catalog.defaultTablePath(table.identifier) + table.copy(storage = table.storage.copy(locationUri = Some(loc))) + } else { + // 1. We are defining a new external table + // 2. It's a managed table which already has the location populated. This can happen in DSV2 + // CTAS flow. + table + } + + val isManagedTable = tableWithLocation.tableType == CatalogTableType.MANAGED + val tableLocation = new Path(tableWithLocation.location) + val gpuDeltaLog = GpuDeltaLog.forTable(sparkSession, tableLocation, rapidsConf) + val hadoopConf = gpuDeltaLog.deltaLog.newDeltaHadoopConf() + val fs = tableLocation.getFileSystem(hadoopConf) + val options = new DeltaOptions(table.storage.properties, sparkSession.sessionState.conf) + var result: Seq[Row] = Nil + + recordDeltaOperation(gpuDeltaLog.deltaLog, "delta.ddl.createTable") { + val txn = gpuDeltaLog.startTransaction() + val opStartTs = System.currentTimeMillis() + if (query.isDefined) { + // If the mode is Ignore or ErrorIfExists, the table must not exist, or we would return + // earlier. And the data should not exist either, to match the behavior of + // Ignore/ErrorIfExists mode. This means the table path should not exist or is empty. + if (mode == SaveMode.Ignore || mode == SaveMode.ErrorIfExists) { + assert(!tableExists) + // We may have failed a previous write. The retry should still succeed even if we have + // garbage data + if (txn.readVersion > -1 || !fs.exists(gpuDeltaLog.deltaLog.logPath)) { + assertPathEmpty(hadoopConf, tableWithLocation) + } + } + // We are either appending/overwriting with saveAsTable or creating a new table with CTAS or + // we are creating a table as part of a RunnableCommand + query.get match { + case writer: WriteIntoDelta => + // In the V2 Writer, methods like "replace" and "createOrReplace" implicitly mean that + // the metadata should be changed. This wasn't the behavior for DataFrameWriterV1. + if (!isV1Writer) { + replaceMetadataIfNecessary( + txn, tableWithLocation, options, writer.data.schema.asNullable) + } + val actions = writer.write(txn, sparkSession) + val op = getOperation(txn.metadata, isManagedTable, Some(options)) + txn.commit(actions, op) + case cmd: RunnableCommand => + result = cmd.run(sparkSession) + case other => + // When using V1 APIs, the `other` plan is not yet optimized, therefore, it is safe + // to once again go through analysis + val data = Dataset.ofRows(sparkSession, other) + + // In the V2 Writer, methods like "replace" and "createOrReplace" implicitly mean that + // the metadata should be changed. This wasn't the behavior for DataFrameWriterV1. + if (!isV1Writer) { + replaceMetadataIfNecessary( + txn, tableWithLocation, options, other.schema.asNullable) + } + + val actions = WriteIntoDelta( + deltaLog = gpuDeltaLog.deltaLog, + mode = mode, + options, + partitionColumns = table.partitionColumnNames, + configuration = tableWithLocation.properties + ("comment" -> table.comment.orNull), + data = data).write(txn, sparkSession) + + val op = getOperation(txn.metadata, isManagedTable, Some(options)) + txn.commit(actions, op) + } + } else { + def createTransactionLogOrVerify(): Unit = { + if (isManagedTable) { + // When creating a managed table, the table path should not exist or is empty, or + // users would be surprised to see the data, or see the data directory being dropped + // after the table is dropped. + assertPathEmpty(hadoopConf, tableWithLocation) + } + + // This is either a new table, or, we never defined the schema of the table. While it is + // unexpected that `txn.metadata.schema` to be empty when txn.readVersion >= 0, we still + // guard against it, in case of checkpoint corruption bugs. + val noExistingMetadata = txn.readVersion == -1 || txn.metadata.schema.isEmpty + if (noExistingMetadata) { + assertTableSchemaDefined(fs, tableLocation, tableWithLocation, txn, sparkSession) + assertPathEmpty(hadoopConf, tableWithLocation) + // This is a user provided schema. + // Doesn't come from a query, Follow nullability invariants. + val newMetadata = getProvidedMetadata(tableWithLocation, table.schema.json) + txn.updateMetadataForNewTable(newMetadata) + + val op = getOperation(newMetadata, isManagedTable, None) + txn.commit(Nil, op) + } else { + verifyTableMetadata(txn, tableWithLocation) + } + } + // We are defining a table using the Create or Replace Table statements. + operation match { + case TableCreationModes.Create => + require(!tableExists, "Can't recreate a table when it exists") + createTransactionLogOrVerify() + + case TableCreationModes.CreateOrReplace if !tableExists => + // If the table doesn't exist, CREATE OR REPLACE must provide a schema + if (tableWithLocation.schema.isEmpty) { + throw DeltaErrors.schemaNotProvidedException + } + createTransactionLogOrVerify() + case _ => + // When the operation is a REPLACE or CREATE OR REPLACE, then the schema shouldn't be + // empty, since we'll use the entry to replace the schema + if (tableWithLocation.schema.isEmpty) { + throw DeltaErrors.schemaNotProvidedException + } + // We need to replace + replaceMetadataIfNecessary(txn, tableWithLocation, options, tableWithLocation.schema) + // Truncate the table + val operationTimestamp = System.currentTimeMillis() + val removes = txn.filterFiles().map(_.removeWithTimestamp(operationTimestamp)) + val op = getOperation(txn.metadata, isManagedTable, None) + txn.commit(removes, op) + } + } + + // We would have failed earlier on if we couldn't ignore the existence of the table + // In addition, we just might using saveAsTable to append to the table, so ignore the creation + // if it already exists. + // Note that someone may have dropped and recreated the table in a separate location in the + // meantime... Unfortunately we can't do anything there at the moment, because Hive sucks. + logInfo(s"Table is path-based table: $tableByPath. Update catalog with mode: $operation") + updateCatalog( + sparkSession, + tableWithLocation, + gpuDeltaLog.deltaLog.update(checkIfUpdatedSinceTs = Some(opStartTs)), + txn) + + result + } + } + + private def getProvidedMetadata(table: CatalogTable, schemaString: String): Metadata = { + Metadata( + description = table.comment.orNull, + schemaString = schemaString, + partitionColumns = table.partitionColumnNames, + configuration = table.properties, + createdTime = Some(System.currentTimeMillis())) + } + + private def assertPathEmpty( + hadoopConf: Configuration, + tableWithLocation: CatalogTable): Unit = { + val path = new Path(tableWithLocation.location) + val fs = path.getFileSystem(hadoopConf) + // Verify that the table location associated with CREATE TABLE doesn't have any data. Note that + // we intentionally diverge from this behavior w.r.t regular datasource tables (that silently + // overwrite any previous data) + if (fs.exists(path) && fs.listStatus(path).nonEmpty) { + throw DeltaErrors.createTableWithNonEmptyLocation( + tableWithLocation.identifier.toString, + tableWithLocation.location.toString) + } + } + + private def assertTableSchemaDefined( + fs: FileSystem, + path: Path, + table: CatalogTable, + txn: OptimisticTransaction, + sparkSession: SparkSession): Unit = { + // If we allow creating an empty schema table and indeed the table is new, we just need to + // make sure: + // 1. txn.readVersion == -1 to read a new table + // 2. for external tables: path must either doesn't exist or is completely empty + val allowCreatingTableWithEmptySchema = sparkSession.sessionState + .conf.getConf(DeltaSQLConf.DELTA_ALLOW_CREATE_EMPTY_SCHEMA_TABLE) && txn.readVersion == -1 + + // Users did not specify the schema. We expect the schema exists in Delta. + if (table.schema.isEmpty) { + if (table.tableType == CatalogTableType.EXTERNAL) { + if (fs.exists(path) && fs.listStatus(path).nonEmpty) { + throw DeltaErrors.createExternalTableWithoutLogException( + path, table.identifier.quotedString, sparkSession) + } else { + if (allowCreatingTableWithEmptySchema) return + throw DeltaErrors.createExternalTableWithoutSchemaException( + path, table.identifier.quotedString, sparkSession) + } + } else { + if (allowCreatingTableWithEmptySchema) return + throw DeltaErrors.createManagedTableWithoutSchemaException( + table.identifier.quotedString, sparkSession) + } + } + } + + /** + * Verify against our transaction metadata that the user specified the right metadata for the + * table. + */ + private def verifyTableMetadata( + txn: OptimisticTransaction, + tableDesc: CatalogTable): Unit = { + val existingMetadata = txn.metadata + val path = new Path(tableDesc.location) + + // The delta log already exists. If they give any configuration, we'll make sure it all matches. + // Otherwise we'll just go with the metadata already present in the log. + // The schema compatibility checks will be made in `WriteIntoDelta` for CreateTable + // with a query + if (txn.readVersion > -1) { + if (tableDesc.schema.nonEmpty) { + // We check exact alignment on create table if everything is provided + // However, if in column mapping mode, we can safely ignore the related metadata fields in + // existing metadata because new table desc will not have related metadata assigned yet + val differences = SchemaUtils.reportDifferences( + DeltaColumnMapping.dropColumnMappingMetadata(existingMetadata.schema), + tableDesc.schema) + if (differences.nonEmpty) { + throw DeltaErrors.createTableWithDifferentSchemaException( + path, tableDesc.schema, existingMetadata.schema, differences) + } + } + + // If schema is specified, we must make sure the partitioning matches, even the partitioning + // is not specified. + if (tableDesc.schema.nonEmpty && + tableDesc.partitionColumnNames != existingMetadata.partitionColumns) { + throw DeltaErrors.createTableWithDifferentPartitioningException( + path, tableDesc.partitionColumnNames, existingMetadata.partitionColumns) + } + + if (tableDesc.properties.nonEmpty && tableDesc.properties != existingMetadata.configuration) { + throw DeltaErrors.createTableWithDifferentPropertiesException( + path, tableDesc.properties, existingMetadata.configuration) + } + } + } + + /** + * Based on the table creation operation, and parameters, we can resolve to different operations. + * A lot of this is needed for legacy reasons in Databricks Runtime. + * @param metadata The table metadata, which we are creating or replacing + * @param isManagedTable Whether we are creating or replacing a managed table + * @param options Write options, if this was a CTAS/RTAS + */ + private def getOperation( + metadata: Metadata, + isManagedTable: Boolean, + options: Option[DeltaOptions]): DeltaOperations.Operation = operation match { + // This is legacy saveAsTable behavior in Databricks Runtime + case TableCreationModes.Create if existingTableOpt.isDefined && query.isDefined => + DeltaOperations.Write(mode, Option(table.partitionColumnNames), options.get.replaceWhere, + options.flatMap(_.userMetadata)) + + // DataSourceV2 table creation + // CREATE TABLE (non-DataFrameWriter API) doesn't have options syntax + // (userMetadata uses SQLConf in this case) + case TableCreationModes.Create => + DeltaOperations.CreateTable(metadata, isManagedTable, query.isDefined) + + // DataSourceV2 table replace + // REPLACE TABLE (non-DataFrameWriter API) doesn't have options syntax + // (userMetadata uses SQLConf in this case) + case TableCreationModes.Replace => + DeltaOperations.ReplaceTable(metadata, isManagedTable, orCreate = false, query.isDefined) + + // Legacy saveAsTable with Overwrite mode + case TableCreationModes.CreateOrReplace if options.exists(_.replaceWhere.isDefined) => + DeltaOperations.Write(mode, Option(table.partitionColumnNames), options.get.replaceWhere, + options.flatMap(_.userMetadata)) + + // New DataSourceV2 saveAsTable with overwrite mode behavior + case TableCreationModes.CreateOrReplace => + DeltaOperations.ReplaceTable(metadata, isManagedTable, orCreate = true, query.isDefined, + options.flatMap(_.userMetadata)) + } + + /** + * Similar to getOperation, here we disambiguate the catalog alterations we need to do based + * on the table operation, and whether we have reached here through legacy code or DataSourceV2 + * code paths. + */ + private def updateCatalog( + spark: SparkSession, + table: CatalogTable, + snapshot: Snapshot, + txn: OptimisticTransaction): Unit = { + val cleaned = cleanupTableDefinition(table, snapshot) + operation match { + case _ if tableByPath => // do nothing with the metastore if this is by path + case TableCreationModes.Create => + spark.sessionState.catalog.createTable( + cleaned, + ignoreIfExists = existingTableOpt.isDefined, + validateLocation = false) + case TableCreationModes.Replace | TableCreationModes.CreateOrReplace + if existingTableOpt.isDefined => + spark.sessionState.catalog.alterTable(table) + case TableCreationModes.Replace => + val ident = Identifier.of(table.identifier.database.toArray, table.identifier.table) + throw DeltaErrors.cannotReplaceMissingTableException(ident) + case TableCreationModes.CreateOrReplace => + spark.sessionState.catalog.createTable( + cleaned, + ignoreIfExists = false, + validateLocation = false) + } + } + + /** Clean up the information we pass on to store in the catalog. */ + private def cleanupTableDefinition(table: CatalogTable, snapshot: Snapshot): CatalogTable = { + // These actually have no effect on the usability of Delta, but feature flagging legacy + // behavior for now + val storageProps = if (conf.getConf(DeltaSQLConf.DELTA_LEGACY_STORE_WRITER_OPTIONS_AS_PROPS)) { + // Legacy behavior + table.storage + } else { + table.storage.copy(properties = Map.empty) + } + + table.copy( + schema = new StructType(), + properties = Map.empty, + partitionColumnNames = Nil, + // Remove write specific options when updating the catalog + storage = storageProps, + tracksPartitionsInCatalog = true) + } + + /** + * With DataFrameWriterV2, methods like `replace()` or `createOrReplace()` mean that the + * metadata of the table should be replaced. If overwriteSchema=false is provided with these + * methods, then we will verify that the metadata match exactly. + */ + private def replaceMetadataIfNecessary( + txn: OptimisticTransaction, + tableDesc: CatalogTable, + options: DeltaOptions, + schema: StructType): Unit = { + val isReplace = (operation == TableCreationModes.CreateOrReplace || + operation == TableCreationModes.Replace) + // If a user explicitly specifies not to overwrite the schema, during a replace, we should + // tell them that it's not supported + val dontOverwriteSchema = options.options.contains(DeltaOptions.OVERWRITE_SCHEMA_OPTION) && + !options.canOverwriteSchema + if (isReplace && dontOverwriteSchema) { + throw DeltaErrors.illegalUsageException(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "replacing") + } + if (txn.readVersion > -1L && isReplace && !dontOverwriteSchema) { + // When a table already exists, and we're using the DataFrameWriterV2 API to replace + // or createOrReplace a table, we blindly overwrite the metadata. + txn.updateMetadataForNewTable(getProvidedMetadata(table, schema.json)) + } + } + + /** + * Horrible hack to differentiate between DataFrameWriterV1 and V2 so that we can decide + * what to do with table metadata. In DataFrameWriterV1, mode("overwrite").saveAsTable, + * behaves as a CreateOrReplace table, but we have asked for "overwriteSchema" as an + * explicit option to overwrite partitioning or schema information. With DataFrameWriterV2, + * the behavior asked for by the user is clearer: .createOrReplace(), which means that we + * should overwrite schema and/or partitioning. Therefore we have this hack. + */ + private def isV1Writer: Boolean = { + Thread.currentThread().getStackTrace.exists(_.toString.contains( + classOf[DataFrameWriter[_]].getCanonicalName + ".")) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeleteCommand.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeleteCommand.scala new file mode 100644 index 00000000000..6c82a910905 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeleteCommand.scala @@ -0,0 +1,377 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from DeleteCommand.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import com.databricks.sql.transaction.tahoe.{DeltaConfigs, DeltaLog, DeltaOperations, DeltaTableUtils, DeltaUDF, OptimisticTransaction} +import com.databricks.sql.transaction.tahoe.DeltaCommitTag._ +import com.databricks.sql.transaction.tahoe.RowTracking +import com.databricks.sql.transaction.tahoe.actions.{AddCDCFile, FileAction} +import com.databricks.sql.transaction.tahoe.commands.{DeleteCommandMetrics, DeleteMetric, DeltaCommand, DMLUtils} +import com.databricks.sql.transaction.tahoe.commands.MergeIntoCommandBase.totalBytesAndDistinctPartitionValues +import com.databricks.sql.transaction.tahoe.files.TahoeBatchFileIndex +import com.databricks.sql.transaction.tahoe.rapids.GpuDeleteCommand.{rewritingFilesMsg, FINDING_TOUCHED_FILES_MSG} +import com.nvidia.spark.rapids.delta.GpuDeltaMetricUpdateUDF + +import org.apache.spark.SparkContext +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, EqualNullSafe, Expression, If, Literal, Not} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.functions.input_file_name +import org.apache.spark.sql.types.LongType + +/** + * GPU version of Delta Lake DeleteCommand. + * + * Performs a Delete based on the search condition + * + * Algorithm: + * 1) Scan all the files and determine which files have + * the rows that need to be deleted. + * 2) Traverse the affected files and rebuild the touched files. + * 3) Use the Delta protocol to atomically write the remaining rows to new files and remove + * the affected files that are identified in step 1. + */ +case class GpuDeleteCommand( + gpuDeltaLog: GpuDeltaLog, + target: LogicalPlan, + condition: Option[Expression]) + extends LeafRunnableCommand with DeltaCommand with DeleteCommandMetrics { + + override def innerChildren: Seq[QueryPlan[_]] = Seq(target) + + override val output: Seq[Attribute] = Seq(AttributeReference("num_affected_rows", LongType)()) + + @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() + + // DeleteCommandMetrics does not include deletion vector metrics, so add them here because + // the commit command needs to collect these metrics for inclusion in the delta log event + override lazy val metrics = createMetrics ++ Map( + "numDeletionVectorsAdded" -> SQLMetrics.createMetric(sc, "number of deletion vectors added."), + "numDeletionVectorsRemoved" -> + SQLMetrics.createMetric(sc, "number of deletion vectors removed."), + "numDeletionVectorsUpdated" -> + SQLMetrics.createMetric(sc, "number of deletion vectors updated.") + ) + + final override def run(sparkSession: SparkSession): Seq[Row] = { + val deltaLog = gpuDeltaLog.deltaLog + recordDeltaOperation(gpuDeltaLog.deltaLog, "delta.dml.delete") { + gpuDeltaLog.withNewTransaction { txn => + DeltaLog.assertRemovable(txn.snapshot) + val deleteCommitTags = performDelete(sparkSession, deltaLog, txn) + val deleteActions = deleteCommitTags.actions + if (deleteActions.nonEmpty) { + txn.commitIfNeeded(deleteActions, DeltaOperations.Delete(condition.toSeq), + deleteCommitTags.stringTags) + } + } + // Re-cache all cached plans(including this relation itself, if it's cached) that refer to + // this data source relation. + sparkSession.sharedState.cacheManager.recacheByPlan(sparkSession, target) + } + + // Adjust for deletes at partition boundaries. Deletes at partition boundaries is a metadata + // operation, therefore we don't actually have any information around how many rows were deleted + // While this info may exist in the file statistics, it's not guaranteed that we have these + // statistics. To avoid any performance regressions, we currently just return a -1 in such cases + if (metrics("numRemovedFiles").value > 0 && metrics("numDeletedRows").value == 0) { + Seq(Row(-1L)) + } else { + Seq(Row(metrics("numDeletedRows").value)) + } + } + + def performDelete( + sparkSession: SparkSession, + deltaLog: DeltaLog, + txn: OptimisticTransaction): DMLUtils.TaggedCommitData = { + import com.databricks.sql.transaction.tahoe.implicits._ + + var numRemovedFiles: Long = 0 + var numAddedFiles: Long = 0 + var numAddedChangeFiles: Long = 0 + var scanTimeMs: Long = 0 + var rewriteTimeMs: Long = 0 + var numBytesAdded: Long = 0 + var changeFileBytes: Long = 0 + var numBytesRemoved: Long = 0 + var numFilesBeforeSkipping: Long = 0 + var numBytesBeforeSkipping: Long = 0 + var numFilesAfterSkipping: Long = 0 + var numBytesAfterSkipping: Long = 0 + var numPartitionsAfterSkipping: Option[Long] = None + var numPartitionsRemovedFrom: Option[Long] = None + var numPartitionsAddedTo: Option[Long] = None + var numDeletedRows: Option[Long] = None + var numCopiedRows: Option[Long] = None + + val startTime = System.nanoTime() + val numFilesTotal = txn.snapshot.numOfFiles + + val deleteActions: Seq[FileAction] = condition match { + case None => + // Case 1: Delete the whole table if the condition is true + val allFiles = txn.filterFiles(Nil) + + numRemovedFiles = allFiles.size + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + val (numBytes, numPartitions) = totalBytesAndDistinctPartitionValues(allFiles) + numBytesRemoved = numBytes + numFilesBeforeSkipping = numRemovedFiles + numBytesBeforeSkipping = numBytes + numFilesAfterSkipping = numRemovedFiles + numBytesAfterSkipping = numBytes + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsAfterSkipping = Some(numPartitions) + numPartitionsRemovedFrom = Some(numPartitions) + numPartitionsAddedTo = Some(0) + } + val operationTimestamp = System.currentTimeMillis() + allFiles.map(_.removeWithTimestamp(operationTimestamp)) + case Some(cond) => + val (metadataPredicates, otherPredicates) = + DeltaTableUtils.splitMetadataAndDataPredicates( + cond, txn.metadata.partitionColumns, sparkSession) + + numFilesBeforeSkipping = txn.snapshot.numOfFiles + numBytesBeforeSkipping = txn.snapshot.sizeInBytes + + if (otherPredicates.isEmpty) { + // Case 2: The condition can be evaluated using metadata only. + // Delete a set of files without the need of scanning any data files. + val operationTimestamp = System.currentTimeMillis() + val candidateFiles = txn.filterFiles(metadataPredicates) + + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + numRemovedFiles = candidateFiles.size + numBytesRemoved = candidateFiles.map(_.size).sum + numFilesAfterSkipping = candidateFiles.size + val (numCandidateBytes, numCandidatePartitions) = + totalBytesAndDistinctPartitionValues(candidateFiles) + numBytesAfterSkipping = numCandidateBytes + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsAfterSkipping = Some(numCandidatePartitions) + numPartitionsRemovedFrom = Some(numCandidatePartitions) + numPartitionsAddedTo = Some(0) + } + candidateFiles.map(_.removeWithTimestamp(operationTimestamp)) + } else { + // Case 3: Delete the rows based on the condition. + val candidateFiles = txn.filterFiles(metadataPredicates ++ otherPredicates) + + numFilesAfterSkipping = candidateFiles.size + val (numCandidateBytes, numCandidatePartitions) = + totalBytesAndDistinctPartitionValues(candidateFiles) + numBytesAfterSkipping = numCandidateBytes + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsAfterSkipping = Some(numCandidatePartitions) + } + + val nameToAddFileMap = generateCandidateFileMap(deltaLog.dataPath, candidateFiles) + + val fileIndex = new TahoeBatchFileIndex( + sparkSession, "delete", candidateFiles, deltaLog, deltaLog.dataPath, txn.snapshot) + // Keep everything from the resolved target except a new TahoeFileIndex + // that only involves the affected files instead of all files. + val newTarget = DeltaTableUtils.replaceFileIndex(target, fileIndex) + val data = Dataset.ofRows(sparkSession, newTarget) + val deletedRowCount = metrics("numDeletedRows") + val deletedRowUdf = DeltaUDF.boolean { + new GpuDeltaMetricUpdateUDF(deletedRowCount) + }.asNondeterministic() + val filesToRewrite = + withStatusCode("DELTA", FINDING_TOUCHED_FILES_MSG) { + if (candidateFiles.isEmpty) { + Array.empty[String] + } else { + data.filter(new Column(cond)) + .select(input_file_name()) + .filter(deletedRowUdf()) + .distinct() + .as[String] + .collect() + } + } + + numRemovedFiles = filesToRewrite.length + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + if (filesToRewrite.isEmpty) { + // Case 3.1: no row matches and no delete will be triggered + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsRemovedFrom = Some(0) + numPartitionsAddedTo = Some(0) + } + Nil + } else { + // Case 3.2: some files need an update to remove the deleted files + // Do the second pass and just read the affected files + val baseRelation = buildBaseRelation( + sparkSession, txn, "delete", deltaLog.dataPath, filesToRewrite, nameToAddFileMap) + // Keep everything from the resolved target except a new TahoeFileIndex + // that only involves the affected files instead of all files. + val newTarget = DeltaTableUtils.replaceFileIndex(target, baseRelation.location) + val targetDF = Dataset.ofRows(sparkSession, newTarget) + val filterCond = Not(EqualNullSafe(cond, Literal.TrueLiteral)) + val rewrittenActions = rewriteFiles(txn, targetDF, filterCond, filesToRewrite.length) + val (changeFiles, rewrittenFiles) = rewrittenActions + .partition(_.isInstanceOf[AddCDCFile]) + numAddedFiles = rewrittenFiles.size + val removedFiles = filesToRewrite.map(f => + getTouchedFile(deltaLog.dataPath, f, nameToAddFileMap)) + val (removedBytes, removedPartitions) = + totalBytesAndDistinctPartitionValues(removedFiles) + numBytesRemoved = removedBytes + val (rewrittenBytes, rewrittenPartitions) = + totalBytesAndDistinctPartitionValues(rewrittenFiles) + numBytesAdded = rewrittenBytes + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsRemovedFrom = Some(removedPartitions) + numPartitionsAddedTo = Some(rewrittenPartitions) + } + numAddedChangeFiles = changeFiles.size + changeFileBytes = changeFiles.collect { case f: AddCDCFile => f.size }.sum + rewriteTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 - scanTimeMs + numDeletedRows = Some(metrics("numDeletedRows").value) + numCopiedRows = Some(metrics("numTouchedRows").value - metrics("numDeletedRows").value) + + val operationTimestamp = System.currentTimeMillis() + removeFilesFromPaths(deltaLog, nameToAddFileMap, filesToRewrite, + operationTimestamp) ++ rewrittenActions + } + } + } + metrics("numRemovedFiles").set(numRemovedFiles) + metrics("numAddedFiles").set(numAddedFiles) + val executionTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + metrics("executionTimeMs").set(executionTimeMs) + metrics("scanTimeMs").set(scanTimeMs) + metrics("rewriteTimeMs").set(rewriteTimeMs) + metrics("numAddedChangeFiles").set(numAddedChangeFiles) + metrics("changeFileBytes").set(changeFileBytes) + metrics("numAddedBytes").set(numBytesAdded) + metrics("numRemovedBytes").set(numBytesRemoved) + metrics("numFilesBeforeSkipping").set(numFilesBeforeSkipping) + metrics("numBytesBeforeSkipping").set(numBytesBeforeSkipping) + metrics("numFilesAfterSkipping").set(numFilesAfterSkipping) + metrics("numBytesAfterSkipping").set(numBytesAfterSkipping) + numPartitionsAfterSkipping.foreach(metrics("numPartitionsAfterSkipping").set) + numPartitionsAddedTo.foreach(metrics("numPartitionsAddedTo").set) + numPartitionsRemovedFrom.foreach(metrics("numPartitionsRemovedFrom").set) + numCopiedRows.foreach(metrics("numCopiedRows").set) + metrics("numDeletionVectorsAdded").set(0) + metrics("numDeletionVectorsRemoved").set(0) + metrics("numDeletionVectorsUpdated").set(0) + txn.registerSQLMetrics(sparkSession, metrics) + // This is needed to make the SQL metrics visible in the Spark UI + val executionId = sparkSession.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates( + sparkSession.sparkContext, executionId, metrics.values.toSeq) + + recordDeltaEvent( + deltaLog, + "delta.dml.delete.stats", + data = DeleteMetric( + condition = condition.map(_.sql).getOrElse("true"), + numFilesTotal, + numFilesAfterSkipping, + numAddedFiles, + numRemovedFiles, + numAddedFiles, + numAddedChangeFiles = numAddedChangeFiles, + numFilesBeforeSkipping, + numBytesBeforeSkipping, + numFilesAfterSkipping, + numBytesAfterSkipping, + numPartitionsAfterSkipping, + numPartitionsAddedTo, + numPartitionsRemovedFrom, + numCopiedRows, + numDeletedRows, + numBytesAdded, + numBytesRemoved, + changeFileBytes = changeFileBytes, + scanTimeMs, + rewriteTimeMs, + // We don't support deletion vectors + numDeletionVectorsAdded = 0, + numDeletionVectorsRemoved = 0, + numDeletionVectorsUpdated = 0) + + ) + + DMLUtils.TaggedCommitData(deleteActions) + .withTag(PreservedRowTrackingTag, RowTracking.isEnabled(txn.protocol, txn.metadata)) + .withTag(NoRowsCopiedTag, metrics("numCopiedRows").value == 0) + } + + /** + * Returns the list of `AddFile`s and `AddCDCFile`s that have been re-written. + */ + private def rewriteFiles( + txn: OptimisticTransaction, + baseData: DataFrame, + filterCondition: Expression, + numFilesToRewrite: Long): Seq[FileAction] = { + val shouldWriteCdc = DeltaConfigs.CHANGE_DATA_FEED.fromMetaData(txn.metadata) + + // number of total rows that we have seen / are either copying or deleting (sum of both). + val numTouchedRows = metrics("numTouchedRows") + val numTouchedRowsUdf = DeltaUDF.boolean { + new GpuDeltaMetricUpdateUDF(numTouchedRows) + }.asNondeterministic() + + withStatusCode( + "DELTA", rewritingFilesMsg(numFilesToRewrite)) { + val dfToWrite = if (shouldWriteCdc) { + import com.databricks.sql.transaction.tahoe.commands.cdc.CDCReader._ + // The logic here ends up being surprisingly elegant, with all source rows ending up in + // the output. Recall that we flipped the user-provided delete condition earlier, before the + // call to `rewriteFiles`. All rows which match this latest `filterCondition` are retained + // as table data, while all rows which don't match are removed from the rewritten table data + // but do get included in the output as CDC events. + baseData + .filter(numTouchedRowsUdf()) + .withColumn( + CDC_TYPE_COLUMN_NAME, + new Column(If(filterCondition, CDC_TYPE_NOT_CDC, CDC_TYPE_DELETE)) + ) + } else { + baseData + .filter(numTouchedRowsUdf()) + .filter(new Column(filterCondition)) + } + + txn.writeFiles(dfToWrite) + } + } +} + +object GpuDeleteCommand { + val FINDING_TOUCHED_FILES_MSG: String = "Finding files to rewrite for DELETE operation" + + def rewritingFilesMsg(numFilesToRewrite: Long): String = + s"Rewriting $numFilesToRewrite files for DELETE operation" +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaCatalog.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaCatalog.scala new file mode 100644 index 00000000000..8c62f4f3fd7 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaCatalog.scala @@ -0,0 +1,218 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from DeltaDataSource.scala in the + * Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import java.util + +import com.databricks.sql.transaction.tahoe.{DeltaConfigs, DeltaErrors} +import com.databricks.sql.transaction.tahoe.commands.TableCreationModes +import com.databricks.sql.transaction.tahoe.metering.DeltaLogging +import com.databricks.sql.transaction.tahoe.sources.DeltaSourceUtils +import com.nvidia.spark.rapids.RapidsConf + +import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, Table} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.types.StructType + +class GpuDeltaCatalog( + override val cpuCatalog: StagingTableCatalog, + override val rapidsConf: RapidsConf) + extends GpuDeltaCatalogBase with SupportsPathIdentifier with DeltaLogging { + + override protected def buildGpuCreateDeltaTableCommand( + rapidsConf: RapidsConf, + table: CatalogTable, + existingTableOpt: Option[CatalogTable], + mode: SaveMode, + query: Option[LogicalPlan], + operation: TableCreationModes.CreationMode, + tableByPath: Boolean): LeafRunnableCommand = { + GpuCreateDeltaTableCommand( + table, + existingTableOpt, + mode, + query, + operation, + tableByPath = tableByPath + )(rapidsConf) + } + + override protected def getExistingTableIfExists(table: TableIdentifier): Option[CatalogTable] = { + // If this is a path identifier, we cannot return an existing CatalogTable. The Create command + // will check the file system itself + if (isPathIdentifier(table)) return None + val tableExists = catalog.tableExists(table) + if (tableExists) { + val oldTable = catalog.getTableMetadata(table) + if (oldTable.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( + s"$table is a view. You may not write data into a view.") + } + if (!DeltaSourceUtils.isDeltaTable(oldTable.provider)) { + throw new AnalysisException(s"$table is not a Delta table. Please drop this " + + "table first if you would like to recreate it with Delta Lake.") + } + Some(oldTable) + } else { + None + } + } + + override protected def verifyTableAndSolidify( + tableDesc: CatalogTable, + query: Option[LogicalPlan]): CatalogTable = { + + if (tableDesc.bucketSpec.isDefined) { + throw DeltaErrors.operationNotSupportedException("Bucketing", tableDesc.identifier) + } + + val schema = query.map { plan => + assert(tableDesc.schema.isEmpty, "Can't specify table schema in CTAS.") + plan.schema.asNullable + }.getOrElse(tableDesc.schema) + + PartitioningUtils.validatePartitionColumn( + schema, + tableDesc.partitionColumnNames, + caseSensitive = false) // Delta is case insensitive + + val validatedConfigurations = DeltaConfigs.validateConfigurations(tableDesc.properties) + + val db = tableDesc.identifier.database.getOrElse(catalog.getCurrentDatabase) + val tableIdentWithDB = tableDesc.identifier.copy(database = Some(db)) + tableDesc.copy( + identifier = tableIdentWithDB, + schema = schema, + properties = validatedConfigurations) + } + + override protected def createGpuStagedDeltaTableV2( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String], + operation: TableCreationModes.CreationMode): StagedTable = { + new GpuStagedDeltaTableV2WithLogging(ident, schema, partitions, properties, operation) + } + + override def loadTable(ident: Identifier, timestamp: Long): Table = { + cpuCatalog.loadTable(ident, timestamp) + } + + override def loadTable(ident: Identifier, version: String): Table = { + cpuCatalog.loadTable(ident, version) + } + + /** + * Creates a Delta table using GPU for writing the data + * + * @param ident The identifier of the table + * @param schema The schema of the table + * @param partitions The partition transforms for the table + * @param allTableProperties The table properties that configure the behavior of the table or + * provide information about the table + * @param writeOptions Options specific to the write during table creation or replacement + * @param sourceQuery A query if this CREATE request came from a CTAS or RTAS + * @param operation The specific table creation mode, whether this is a + * Create/Replace/Create or Replace + */ + override def createDeltaTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + allTableProperties: util.Map[String, String], + writeOptions: Map[String, String], + sourceQuery: Option[DataFrame], + operation: TableCreationModes.CreationMode + ): Table = recordFrameProfile( + "DeltaCatalog", "createDeltaTable") { + super.createDeltaTable( + ident, + schema, + partitions, + allTableProperties, + writeOptions, + sourceQuery, + operation) + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = + recordFrameProfile("DeltaCatalog", "createTable") { + super.createTable(ident, schema, partitions, properties) + } + + override def stageReplace( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = + recordFrameProfile("DeltaCatalog", "stageReplace") { + super.stageReplace(ident, schema, partitions, properties) + } + + override def stageCreateOrReplace( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = + recordFrameProfile("DeltaCatalog", "stageCreateOrReplace") { + super.stageCreateOrReplace(ident, schema, partitions, properties) + } + + override def stageCreate( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = + recordFrameProfile("DeltaCatalog", "stageCreate") { + super.stageCreate(ident, schema, partitions, properties) + } + + /** + * A staged Delta table, which creates a HiveMetaStore entry and appends data if this was a + * CTAS/RTAS command. We have a ugly way of using this API right now, but it's the best way to + * maintain old behavior compatibility between Databricks Runtime and OSS Delta Lake. + */ + protected class GpuStagedDeltaTableV2WithLogging( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String], + operation: TableCreationModes.CreationMode) + extends GpuStagedDeltaTableV2(ident, schema, partitions, properties, operation) { + + override def commitStagedChanges(): Unit = recordFrameProfile( + "DeltaCatalog", "commitStagedChanges") { + super.commitStagedChanges() + } + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDoAutoCompaction.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDoAutoCompaction.scala new file mode 100644 index 00000000000..9726511ad44 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDoAutoCompaction.scala @@ -0,0 +1,53 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from DoAutoCompaction.scala + * from https://github.com/delta-io/delta/pull/1156 + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.actions.Action +import com.databricks.sql.transaction.tahoe.hooks.PostCommitHook +import com.databricks.sql.transaction.tahoe.metering.DeltaLogging + +import org.apache.spark.sql.SparkSession + +object GpuDoAutoCompaction extends PostCommitHook + with DeltaLogging + with Serializable { + override val name: String = "Triggers compaction if necessary" + + override def run(spark: SparkSession, + txn: OptimisticTransactionImpl, + committedVersion: Long, + postCommitSnapshot: Snapshot, + committedActions: Seq[Action]): Unit = { + val gpuTxn = txn.asInstanceOf[GpuOptimisticTransaction] + val newTxn = new GpuDeltaLog(gpuTxn.deltaLog, gpuTxn.rapidsConf).startTransaction() + // Note: The Databricks AutoCompact PostCommitHook cannot be used here + // (with a GpuOptimisticTransaction). It appears that AutoCompact creates a new transaction, + // thereby circumventing GpuOptimisticTransaction (which intercepts Parquet writes + // to go through the GPU). + new GpuOptimizeExecutor(spark, newTxn, Seq.empty, Seq.empty, committedActions).optimize() + } + + override def handleError(error: Throwable, version: Long): Unit = + throw DeltaErrors.postCommitHookFailedException(this, version, name, error) +} \ No newline at end of file diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuLowShuffleMergeCommand.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuLowShuffleMergeCommand.scala new file mode 100644 index 00000000000..fddebda33bd --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuLowShuffleMergeCommand.scala @@ -0,0 +1,1083 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * This file was derived from MergeIntoCommand.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import java.net.URI +import java.util.concurrent.TimeUnit + +import scala.collection.mutable + +import com.databricks.sql.io.RowIndexFilterType +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.DeltaOperations.MergePredicate +import com.databricks.sql.transaction.tahoe.DeltaParquetFileFormat.DeletionVectorDescriptorWithFilterType +import com.databricks.sql.transaction.tahoe.actions.{AddCDCFile, AddFile, DeletionVectorDescriptor, FileAction} +import com.databricks.sql.transaction.tahoe.commands.DeltaCommand +import com.databricks.sql.transaction.tahoe.rapids.MergeExecutor.{toDeletionVector, totalBytesAndDistinctPartitionValues, FILE_PATH_COL, INCR_METRICS_COL, INCR_METRICS_FIELD, ROW_DROPPED_COL, ROW_DROPPED_FIELD, SOURCE_ROW_PRESENT_COL, SOURCE_ROW_PRESENT_FIELD, TARGET_ROW_PRESENT_COL, TARGET_ROW_PRESENT_FIELD} +import com.databricks.sql.transaction.tahoe.schema.ImplicitMetadataOperation +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.databricks.sql.transaction.tahoe.util.{AnalysisHelper, DeltaFileOperations} +import com.nvidia.spark.rapids.{GpuOverrides, RapidsConf, SparkPlanMeta} +import com.nvidia.spark.rapids.RapidsConf.DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD +import com.nvidia.spark.rapids.delta._ +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormatUtils.{METADATA_ROW_DEL_COL, METADATA_ROW_DEL_FIELD, METADATA_ROW_IDX_COL, METADATA_ROW_IDX_FIELD} +import com.nvidia.spark.rapids.shims.FileSourceScanExecMeta +import org.roaringbitmap.longlong.Roaring64Bitmap + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, CaseWhen, Expression, Literal, NamedExpression, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral +import org.apache.spark.sql.catalyst.plans.logical.{DeltaMergeAction, DeltaMergeIntoClause, DeltaMergeIntoMatchedClause, DeltaMergeIntoMatchedDeleteClause, DeltaMergeIntoMatchedUpdateClause, DeltaMergeIntoNotMatchedBySourceClause, DeltaMergeIntoNotMatchedBySourceDeleteClause, DeltaMergeIntoNotMatchedBySourceUpdateClause, DeltaMergeIntoNotMatchedClause, DeltaMergeIntoNotMatchedInsertClause, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} + +/** + * GPU version of Delta Lake's low shuffle merge implementation. + * + * Performs a merge of a source query/table into a Delta table. + * + * Issues an error message when the ON search_condition of the MERGE statement can match + * a single row from the target table with multiple rows of the source table-reference. + * Different from the original implementation, it optimized writing touched unmodified target files. + * + * Algorithm: + * + * Phase 1: Find the input files in target that are touched by the rows that satisfy + * the condition and verify that no two source rows match with the same target row. + * This is implemented as an inner-join using the given condition. See [[findTouchedFiles]] + * for more details. + * + * Phase 2: Read the touched files again and write new files with updated and/or inserted rows + * without copying unmodified rows. + * + * Phase 3: Read the touched files again and write new files with unmodified rows in target table, + * trying to keep its original order and avoid shuffle as much as possible. + * + * Phase 4: Use the Delta protocol to atomically remove the touched files and add the new files. + * + * @param source Source data to merge from + * @param target Target table to merge into + * @param gpuDeltaLog Delta log to use + * @param condition Condition for a source row to match with a target row + * @param matchedClauses All info related to matched clauses. + * @param notMatchedClauses All info related to not matched clause. + * @param migratedSchema The final schema of the target - may be changed by schema evolution. + */ +case class GpuLowShuffleMergeCommand( + @transient source: LogicalPlan, + @transient target: LogicalPlan, + @transient gpuDeltaLog: GpuDeltaLog, + condition: Expression, + matchedClauses: Seq[DeltaMergeIntoMatchedClause], + notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], + notMatchedBySourceClauses: Seq[DeltaMergeIntoNotMatchedBySourceClause], + migratedSchema: Option[StructType])( + @transient val rapidsConf: RapidsConf) + extends LeafRunnableCommand + with DeltaCommand with PredicateHelper with AnalysisHelper with ImplicitMetadataOperation { + + import SQLMetrics._ + + override val otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) + + override val canMergeSchema: Boolean = conf.getConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE) + override val canOverwriteSchema: Boolean = false + + override val output: Seq[Attribute] = Seq( + AttributeReference("num_affected_rows", LongType)(), + AttributeReference("num_updated_rows", LongType)(), + AttributeReference("num_deleted_rows", LongType)(), + AttributeReference("num_inserted_rows", LongType)()) + + @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() + @transient lazy val targetDeltaLog: DeltaLog = gpuDeltaLog.deltaLog + + override lazy val metrics = Map[String, SQLMetric]( + "numSourceRows" -> createMetric(sc, "number of source rows"), + "numSourceRowsInSecondScan" -> + createMetric(sc, "number of source rows (during repeated scan)"), + "numTargetRowsCopied" -> createMetric(sc, "number of target rows rewritten unmodified"), + "numTargetRowsInserted" -> createMetric(sc, "number of inserted rows"), + "numTargetRowsUpdated" -> createMetric(sc, "number of updated rows"), + "numTargetRowsDeleted" -> createMetric(sc, "number of deleted rows"), + "numTargetRowsMatchedUpdated" -> createMetric(sc, "number of target rows updated when matched"), + "numTargetRowsMatchedDeleted" -> createMetric(sc, "number of target rows deleted when matched"), + "numTargetRowsNotMatchedBySourceUpdated" -> createMetric(sc, + "number of target rows updated when not matched by source"), + "numTargetRowsNotMatchedBySourceDeleted" -> createMetric(sc, + "number of target rows deleted when not matched by source"), + "numTargetFilesBeforeSkipping" -> createMetric(sc, "number of target files before skipping"), + "numTargetFilesAfterSkipping" -> createMetric(sc, "number of target files after skipping"), + "numTargetFilesRemoved" -> createMetric(sc, "number of files removed to target"), + "numTargetFilesAdded" -> createMetric(sc, "number of files added to target"), + "numTargetChangeFilesAdded" -> + createMetric(sc, "number of change data capture files generated"), + "numTargetChangeFileBytes" -> + createMetric(sc, "total size of change data capture files generated"), + "numTargetBytesBeforeSkipping" -> createMetric(sc, "number of target bytes before skipping"), + "numTargetBytesAfterSkipping" -> createMetric(sc, "number of target bytes after skipping"), + "numTargetBytesRemoved" -> createMetric(sc, "number of target bytes removed"), + "numTargetBytesAdded" -> createMetric(sc, "number of target bytes added"), + "numTargetPartitionsAfterSkipping" -> + createMetric(sc, "number of target partitions after skipping"), + "numTargetPartitionsRemovedFrom" -> + createMetric(sc, "number of target partitions from which files were removed"), + "numTargetPartitionsAddedTo" -> + createMetric(sc, "number of target partitions to which files were added"), + "executionTimeMs" -> + createMetric(sc, "time taken to execute the entire operation"), + "scanTimeMs" -> + createMetric(sc, "time taken to scan the files for matches"), + "rewriteTimeMs" -> + createMetric(sc, "time taken to rewrite the matched files")) + + /** Whether this merge statement has only a single insert (NOT MATCHED) clause. */ + protected def isSingleInsertOnly: Boolean = matchedClauses.isEmpty && + notMatchedClauses.length == 1 + + override def run(spark: SparkSession): Seq[Row] = { + recordDeltaOperation(targetDeltaLog, "delta.dml.lowshufflemerge") { + val startTime = System.nanoTime() + val result = gpuDeltaLog.withNewTransaction { deltaTxn => + if (target.schema.size != deltaTxn.metadata.schema.size) { + throw DeltaErrors.schemaChangedSinceAnalysis( + atAnalysis = target.schema, latestSchema = deltaTxn.metadata.schema) + } + + if (canMergeSchema) { + updateMetadata( + spark, deltaTxn, migratedSchema.getOrElse(target.schema), + deltaTxn.metadata.partitionColumns, deltaTxn.metadata.configuration, + isOverwriteMode = false, rearrangeOnly = false) + } + + + val (executor, fallback) = { + val context = MergeExecutorContext(this, spark, deltaTxn, rapidsConf) + if (isSingleInsertOnly && spark.conf.get(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED)) { + (new InsertOnlyMergeExecutor(context), false) + } else { + val executor = new LowShuffleMergeExecutor(context) + (executor, executor.shouldFallback()) + } + } + + if (fallback) { + None + } else { + Some(runLowShuffleMerge(spark, startTime, deltaTxn, executor)) + } + } + + result match { + case Some(row) => row + case None => + // We should rollback to normal gpu + new GpuMergeIntoCommand(source, target, gpuDeltaLog, condition, matchedClauses, + notMatchedClauses, notMatchedBySourceClauses, migratedSchema)(rapidsConf) + .run(spark) + } + } + } + + + private def runLowShuffleMerge( + spark: SparkSession, + startTime: Long, + deltaTxn: GpuOptimisticTransactionBase, + mergeExecutor: MergeExecutor): Seq[Row] = { + val deltaActions = mergeExecutor.execute() + // Metrics should be recorded before commit (where they are written to delta logs). + metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) + deltaTxn.registerSQLMetrics(spark, metrics) + + // This is a best-effort sanity check. + if (metrics("numSourceRowsInSecondScan").value >= 0 && + metrics("numSourceRows").value != metrics("numSourceRowsInSecondScan").value) { + log.warn(s"Merge source has ${metrics("numSourceRows").value} rows in initial scan but " + + s"${metrics("numSourceRowsInSecondScan").value} rows in second scan") + if (conf.getConf(DeltaSQLConf.MERGE_FAIL_IF_SOURCE_CHANGED)) { + throw DeltaErrors.sourceNotDeterministicInMergeException(spark) + } + } + + deltaTxn.commit( + deltaActions, + DeltaOperations.Merge( + Option(condition), + matchedClauses.map(DeltaOperations.MergePredicate(_)), + notMatchedClauses.map(DeltaOperations.MergePredicate(_)), + // We do not support notMatchedBySourcePredicates yet and fall back to CPU + // See https://github.com/NVIDIA/spark-rapids/issues/8415 + notMatchedBySourcePredicates = Seq.empty[MergePredicate] + )) + + // Record metrics + val stats = GpuMergeStats.fromMergeSQLMetrics( + metrics, + condition, + matchedClauses, + notMatchedClauses, + deltaTxn.metadata.partitionColumns.nonEmpty) + recordDeltaEvent(targetDeltaLog, "delta.dml.merge.stats", data = stats) + + + spark.sharedState.cacheManager.recacheByPlan(spark, target) + + // This is needed to make the SQL metrics visible in the Spark UI. Also this needs + // to be outside the recordMergeOperation because this method will update some metric. + val executionId = spark.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(spark.sparkContext, executionId, metrics.values.toSeq) + Seq(Row(metrics("numTargetRowsUpdated").value + metrics("numTargetRowsDeleted").value + + metrics("numTargetRowsInserted").value, metrics("numTargetRowsUpdated").value, + metrics("numTargetRowsDeleted").value, metrics("numTargetRowsInserted").value)) + } + + /** + * Execute the given `thunk` and return its result while recording the time taken to do it. + * + * @param sqlMetricName name of SQL metric to update with the time taken by the thunk + * @param thunk the code to execute + */ + def recordMergeOperation[A](sqlMetricName: String)(thunk: => A): A = { + val startTimeNs = System.nanoTime() + val r = thunk + val timeTakenMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + if (sqlMetricName != null && timeTakenMs > 0) { + metrics(sqlMetricName) += timeTakenMs + } + r + } + + /** Expressions to increment SQL metrics */ + def makeMetricUpdateUDF(name: String, deterministic: Boolean = false) + : Expression = { + // only capture the needed metric in a local variable + val metric = metrics(name) + var u = DeltaUDF.boolean(new GpuDeltaMetricUpdateUDF(metric)) + if (!deterministic) { + u = u.asNondeterministic() + } + u.apply().expr + } +} + +/** + * Context merge execution. + */ +case class MergeExecutorContext(cmd: GpuLowShuffleMergeCommand, + spark: SparkSession, + deltaTxn: OptimisticTransaction, + rapidsConf: RapidsConf) + +trait MergeExecutor extends AnalysisHelper with PredicateHelper with Logging { + + val context: MergeExecutorContext + + + /** + * Map to get target output attributes by name. + * The case sensitivity of the map is set accordingly to Spark configuration. + */ + @transient private lazy val targetOutputAttributesMap: Map[String, Attribute] = { + val attrMap: Map[String, Attribute] = context.cmd.target + .outputSet.view + .map(attr => attr.name -> attr).toMap + if (context.cmd.conf.caseSensitiveAnalysis) { + attrMap + } else { + CaseInsensitiveMap(attrMap) + } + } + + def execute(): Seq[FileAction] + + protected def targetOutputCols: Seq[NamedExpression] = { + context.deltaTxn.metadata.schema.map { col => + targetOutputAttributesMap + .get(col.name) + .map { a => + AttributeReference(col.name, col.dataType, col.nullable)(a.exprId) + } + .getOrElse(Alias(Literal(null), col.name)()) + } + } + + /** + * Build a DataFrame using the given `files` that has the same output columns (exprIds) + * as the `target` logical plan, so that existing update/insert expressions can be applied + * on this new plan. + */ + protected def buildTargetDFWithFiles(files: Seq[AddFile]): DataFrame = { + val targetOutputColsMap = { + val colsMap: Map[String, NamedExpression] = targetOutputCols.view + .map(col => col.name -> col).toMap + if (context.cmd.conf.caseSensitiveAnalysis) { + colsMap + } else { + CaseInsensitiveMap(colsMap) + } + } + + val plan = { + // We have to do surgery to use the attributes from `targetOutputCols` to scan the table. + // In cases of schema evolution, they may not be the same type as the original attributes. + val original = + context.deltaTxn.deltaLog.createDataFrame(context.deltaTxn.snapshot, files) + .queryExecution + .analyzed + val transformed = original.transform { + case LogicalRelation(base, _, catalogTbl, isStreaming) => + LogicalRelation( + base, + // We can ignore the new columns which aren't yet AttributeReferences. + targetOutputCols.collect { case a: AttributeReference => a }, + catalogTbl, + isStreaming) + } + + // In case of schema evolution & column mapping, we would also need to rebuild the file + // format because under column mapping, the reference schema within DeltaParquetFileFormat + // that is used to populate metadata needs to be updated + if (context.deltaTxn.metadata.columnMappingMode != NoMapping) { + val updatedFileFormat = context.deltaTxn.deltaLog.fileFormat( + context.deltaTxn.deltaLog.unsafeVolatileSnapshot.protocol, context.deltaTxn.metadata) + DeltaTableUtils.replaceFileFormat(transformed, updatedFileFormat) + } else { + transformed + } + } + + // For each plan output column, find the corresponding target output column (by name) and + // create an alias + val aliases = plan.output.map { + case newAttrib: AttributeReference => + val existingTargetAttrib = targetOutputColsMap.getOrElse(newAttrib.name, + throw new AnalysisException( + s"Could not find ${newAttrib.name} among the existing target output " + + targetOutputCols.mkString(","))).asInstanceOf[AttributeReference] + + if (existingTargetAttrib.exprId == newAttrib.exprId) { + // It's not valid to alias an expression to its own exprId (this is considered a + // non-unique exprId by the analyzer), so we just use the attribute directly. + newAttrib + } else { + Alias(newAttrib, existingTargetAttrib.name)(exprId = existingTargetAttrib.exprId) + } + } + + Dataset.ofRows(context.spark, Project(aliases, plan)) + } + + + /** + * Repartitions the output DataFrame by the partition columns if table is partitioned + * and `merge.repartitionBeforeWrite.enabled` is set to true. + */ + protected def repartitionIfNeeded(df: DataFrame): DataFrame = { + val partitionColumns = context.deltaTxn.metadata.partitionColumns + // TODO: We should remove this method and use optimized write instead, see + // https://github.com/NVIDIA/spark-rapids/issues/10417 + if (partitionColumns.nonEmpty && context.spark.conf.get(DeltaSQLConf + .MERGE_REPARTITION_BEFORE_WRITE)) { + df.repartition(partitionColumns.map(col): _*) + } else { + df + } + } + + protected def sourceDF: DataFrame = { + // UDF to increment metrics + val incrSourceRowCountExpr = context.cmd.makeMetricUpdateUDF("numSourceRows") + Dataset.ofRows(context.spark, context.cmd.source) + .filter(new Column(incrSourceRowCountExpr)) + } + + /** Whether this merge statement has no insert (NOT MATCHED) clause. */ + protected def hasNoInserts: Boolean = context.cmd.notMatchedClauses.isEmpty + + +} + +/** + * This is an optimization of the case when there is no update clause for the merge. + * We perform an left anti join on the source data to find the rows to be inserted. + * + * This will currently only optimize for the case when there is a _single_ notMatchedClause. + */ +class InsertOnlyMergeExecutor(override val context: MergeExecutorContext) extends MergeExecutor { + override def execute(): Seq[FileAction] = { + context.cmd.recordMergeOperation(sqlMetricName = "rewriteTimeMs") { + + // UDFs to update metrics + val incrSourceRowCountExpr = context.cmd.makeMetricUpdateUDF("numSourceRows") + val incrInsertedCountExpr = context.cmd.makeMetricUpdateUDF("numTargetRowsInserted") + + val outputColNames = targetOutputCols.map(_.name) + // we use head here since we know there is only a single notMatchedClause + val outputExprs = context.cmd.notMatchedClauses.head.resolvedActions.map(_.expr) + val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => + new Column(Alias(expr, name)()) + } + + // source DataFrame + val sourceDF = Dataset.ofRows(context.spark, context.cmd.source) + .filter(new Column(incrSourceRowCountExpr)) + .filter(new Column(context.cmd.notMatchedClauses.head.condition + .getOrElse(Literal.TrueLiteral))) + + // Skip data based on the merge condition + val conjunctivePredicates = splitConjunctivePredicates(context.cmd.condition) + val targetOnlyPredicates = + conjunctivePredicates.filter(_.references.subsetOf(context.cmd.target.outputSet)) + val dataSkippedFiles = context.deltaTxn.filterFiles(targetOnlyPredicates) + + // target DataFrame + val targetDF = buildTargetDFWithFiles(dataSkippedFiles) + + val insertDf = sourceDF.join(targetDF, new Column(context.cmd.condition), "leftanti") + .select(outputCols: _*) + .filter(new Column(incrInsertedCountExpr)) + + val newFiles = context.deltaTxn + .writeFiles(repartitionIfNeeded(insertDf, + )) + + // Update metrics + context.cmd.metrics("numTargetFilesBeforeSkipping") += context.deltaTxn.snapshot.numOfFiles + context.cmd.metrics("numTargetBytesBeforeSkipping") += context.deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + context.cmd.metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + context.cmd.metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + context.cmd.metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + context.cmd.metrics("numTargetFilesRemoved") += 0 + context.cmd.metrics("numTargetBytesRemoved") += 0 + context.cmd.metrics("numTargetPartitionsRemovedFrom") += 0 + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + context.cmd.metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + context.cmd.metrics("numTargetBytesAdded") += addedBytes + context.cmd.metrics("numTargetPartitionsAddedTo") += addedPartitions + newFiles + } + } +} + + +/** + * This is an optimized algorithm for merge statement, where we avoid shuffling the unmodified + * target data. + * + * The algorithm is as follows: + * 1. Find touched target files in the target table by joining the source and target data, with + * collecting joined row identifiers as (`__metadata_file_path`, `__metadata_row_idx`) pairs. + * 2. Read the touched files again and write new files with updated and/or inserted rows + * without coping unmodified data from target table, but filtering target table with collected + * rows mentioned above. + * 3. Read the touched files again, filtering unmodified rows with collected row identifiers + * collected in first step, and saving them without shuffle. + */ +class LowShuffleMergeExecutor(override val context: MergeExecutorContext) extends MergeExecutor { + + // We over-count numTargetRowsDeleted when there are multiple matches; + // this is the amount of the overcount, so we can subtract it to get a correct final metric. + private var multipleMatchDeleteOnlyOvercount: Option[Long] = None + + // UDFs to update metrics + private val incrSourceRowCountExpr: Expression = context.cmd. + makeMetricUpdateUDF("numSourceRowsInSecondScan") + private val incrUpdatedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsUpdated") + private val incrUpdatedMatchedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsMatchedUpdated") + private val incrUpdatedNotMatchedBySourceCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceUpdated") + private val incrInsertedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsInserted") + private val incrDeletedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsDeleted") + private val incrDeletedMatchedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsMatchedDeleted") + private val incrDeletedNotMatchedBySourceCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceDeleted") + + private def updateOutput(resolvedActions: Seq[DeltaMergeAction], incrExpr: Expression) + : Seq[Expression] = { + resolvedActions.map(_.expr) :+ + Literal.FalseLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def deleteOutput(incrExpr: Expression): Seq[Expression] = { + targetOutputCols :+ + TrueLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def insertOutput(resolvedActions: Seq[DeltaMergeAction], incrExpr: Expression) + : Seq[Expression] = { + resolvedActions.map(_.expr) :+ + Literal.FalseLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def clauseOutput(clause: DeltaMergeIntoClause): Seq[Expression] = clause match { + case u: DeltaMergeIntoMatchedUpdateClause => + updateOutput(u.resolvedActions, And(incrUpdatedCountExpr, incrUpdatedMatchedCountExpr)) + case _: DeltaMergeIntoMatchedDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedMatchedCountExpr)) + case i: DeltaMergeIntoNotMatchedInsertClause => + insertOutput(i.resolvedActions, incrInsertedCountExpr) + case u: DeltaMergeIntoNotMatchedBySourceUpdateClause => + updateOutput(u.resolvedActions, + And(incrUpdatedCountExpr, incrUpdatedNotMatchedBySourceCountExpr)) + case _: DeltaMergeIntoNotMatchedBySourceDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedNotMatchedBySourceCountExpr)) + } + + private def clauseCondition(clause: DeltaMergeIntoClause): Expression = { + // if condition is None, then expression always evaluates to true + clause.condition.getOrElse(TrueLiteral) + } + + /** + * Though low shuffle merge algorithm performs better than traditional merge algorithm in some + * cases, there are some case we should fallback to traditional merge executor: + * + * 1. Low shuffle merge algorithm requires generating metadata columns such as + * [[METADATA_ROW_IDX_COL]], [[METADATA_ROW_DEL_COL]], which only implemented on + * [[org.apache.spark.sql.rapids.GpuFileSourceScanExec]]. That means we need to fallback to + * this normal executor when [[org.apache.spark.sql.rapids.GpuFileSourceScanExec]] is disabled + * for some reason. + * 2. Low shuffle merge algorithm currently needs to broadcast deletion vector, which may + * introduce extra overhead. It maybe better to fallback to this algorithm when the changeset + * it too large. + */ + def shouldFallback(): Boolean = { + // Trying to detect if we can execute finding touched files. + val touchFilePlanOverrideSucceed = verifyGpuPlan(planForFindingTouchedFiles()) { planMeta => + def check(meta: SparkPlanMeta[SparkPlan]): Boolean = { + meta match { + case scan if scan.isInstanceOf[FileSourceScanExecMeta] => scan + .asInstanceOf[FileSourceScanExecMeta] + .wrapped + .schema + .fieldNames + .contains(METADATA_ROW_IDX_COL) && scan.canThisBeReplaced + case m => m.childPlans.exists(check) + } + } + + check(planMeta) + } + if (!touchFilePlanOverrideSucceed) { + logWarning("Unable to override file scan for low shuffle merge for finding touched files " + + "plan, fallback to tradition merge.") + return true + } + + // Trying to detect if we can execute the merge plan. + val mergePlanOverrideSucceed = verifyGpuPlan(planForMergeExecution(touchedFiles)) { planMeta => + var overrideCount = 0 + def count(meta: SparkPlanMeta[SparkPlan]): Unit = { + meta match { + case scan if scan.isInstanceOf[FileSourceScanExecMeta] => + if (scan.asInstanceOf[FileSourceScanExecMeta] + .wrapped.schema.fieldNames.contains(METADATA_ROW_DEL_COL) && scan.canThisBeReplaced) { + overrideCount += 1 + } + case m => m.childPlans.foreach(count) + } + } + + count(planMeta) + overrideCount == 2 + } + + if (!mergePlanOverrideSucceed) { + logWarning("Unable to override file scan for low shuffle merge for merge plan, fallback to " + + "tradition merge.") + return true + } + + val deletionVectorSize = touchedFiles.values.map(_._1.serializedSizeInBytes()).sum + val maxDelVectorSize = context.rapidsConf + .get(DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD) + if (deletionVectorSize > maxDelVectorSize) { + logWarning( + s"""Low shuffle merge can't be executed because broadcast deletion vector count + |$deletionVectorSize is large than max value $maxDelVectorSize """.stripMargin) + return true + } + + false + } + + private def verifyGpuPlan(input: DataFrame)(checkPlanMeta: SparkPlanMeta[SparkPlan] => Boolean) + : Boolean = { + val overridePlan = GpuOverrides.wrapAndTagPlan(input.queryExecution.sparkPlan, + context.rapidsConf) + checkPlanMeta(overridePlan) + } + + override def execute(): Seq[FileAction] = { + val newFiles = context.cmd.withStatusCode("DELTA", + s"Rewriting ${touchedFiles.size} files and saving modified data") { + val df = planForMergeExecution(touchedFiles) + context.deltaTxn.writeFiles(df) + } + + // Update metrics + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + context.cmd.metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + context.cmd.metrics("numTargetChangeFilesAdded") += newFiles.count(_.isInstanceOf[AddCDCFile]) + context.cmd.metrics("numTargetChangeFileBytes") += newFiles.collect { + case f: AddCDCFile => f.size + } + .sum + context.cmd.metrics("numTargetBytesAdded") += addedBytes + context.cmd.metrics("numTargetPartitionsAddedTo") += addedPartitions + + if (multipleMatchDeleteOnlyOvercount.isDefined) { + // Compensate for counting duplicates during the query. + val actualRowsDeleted = + context.cmd.metrics("numTargetRowsDeleted").value - multipleMatchDeleteOnlyOvercount.get + assert(actualRowsDeleted >= 0) + context.cmd.metrics("numTargetRowsDeleted").set(actualRowsDeleted) + } + + touchedFiles.values.map(_._2).map(_.remove).toSeq ++ newFiles + } + + private lazy val dataSkippedFiles: Seq[AddFile] = { + // Skip data based on the merge condition + val targetOnlyPredicates = splitConjunctivePredicates(context.cmd.condition) + .filter(_.references.subsetOf(context.cmd.target.outputSet)) + context.deltaTxn.filterFiles(targetOnlyPredicates) + } + + private lazy val dataSkippedTargetDF: DataFrame = { + addRowIndexMetaColumn(buildTargetDFWithFiles(dataSkippedFiles)) + } + + private lazy val touchedFiles: Map[String, (Roaring64Bitmap, AddFile)] = this.findTouchedFiles() + + private def planForFindingTouchedFiles(): DataFrame = { + + // Apply inner join to between source and target using the merge condition to find matches + // In addition, we attach two columns + // - METADATA_ROW_IDX column to identify target row in file + // - FILE_PATH_COL the target file name the row is from to later identify the files touched + // by matched rows + val targetDF = dataSkippedTargetDF.withColumn(FILE_PATH_COL, input_file_name()) + + sourceDF.join(targetDF, new Column(context.cmd.condition), "inner") + } + + private def planForMergeExecution(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]) + : DataFrame = { + getModifiedDF(touchedFiles).unionAll(getUnmodifiedDF(touchedFiles)) + } + + /** + * Find the target table files that contain the rows that satisfy the merge condition. This is + * implemented as an inner-join between the source query/table and the target table using + * the merge condition. + */ + private def findTouchedFiles(): Map[String, (Roaring64Bitmap, AddFile)] = + context.cmd.recordMergeOperation(sqlMetricName = "scanTimeMs") { + context.spark.udf.register("row_index_set", udaf(RoaringBitmapUDAF)) + // Process the matches from the inner join to record touched files and find multiple matches + val collectTouchedFiles = planForFindingTouchedFiles() + .select(col(FILE_PATH_COL), col(METADATA_ROW_IDX_COL)) + .groupBy(FILE_PATH_COL) + .agg( + expr(s"row_index_set($METADATA_ROW_IDX_COL) as row_idxes"), + count("*").as("count")) + .collect().map(row => { + val filename = row.getAs[String](FILE_PATH_COL) + val rowIdxSet = row.getAs[RoaringBitmapWrapper]("row_idxes").inner + val count = row.getAs[Long]("count") + (filename, (rowIdxSet, count)) + }) + .toMap + + val duplicateCount = { + val distinctMatchedRowCounts = collectTouchedFiles.values + .map(_._1.getLongCardinality).sum + val allMatchedRowCounts = collectTouchedFiles.values.map(_._2).sum + allMatchedRowCounts - distinctMatchedRowCounts + } + + val hasMultipleMatches = duplicateCount > 0 + + // Throw error if multiple matches are ambiguous or cannot be computed correctly. + val canBeComputedUnambiguously = { + // Multiple matches are not ambiguous when there is only one unconditional delete as + // all the matched row pairs in the 2nd join in `writeAllChanges` will get deleted. + val isUnconditionalDelete = context.cmd.matchedClauses.headOption match { + case Some(DeltaMergeIntoMatchedDeleteClause(None)) => true + case _ => false + } + context.cmd.matchedClauses.size == 1 && isUnconditionalDelete + } + + if (hasMultipleMatches && !canBeComputedUnambiguously) { + throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException(context.spark) + } + + if (hasMultipleMatches) { + // This is only allowed for delete-only queries. + // This query will count the duplicates for numTargetRowsDeleted in Job 2, + // because we count matches after the join and not just the target rows. + // We have to compensate for this by subtracting the duplicates later, + // so we need to record them here. + multipleMatchDeleteOnlyOvercount = Some(duplicateCount) + } + + // Get the AddFiles using the touched file names. + val touchedFileNames = collectTouchedFiles.keys.toSeq + + val nameToAddFileMap = context.cmd.generateCandidateFileMap( + context.cmd.targetDeltaLog.dataPath, + dataSkippedFiles) + + val touchedAddFiles = touchedFileNames.map(f => + context.cmd.getTouchedFile(context.cmd.targetDeltaLog.dataPath, f, nameToAddFileMap)) + .map(f => (DeltaFileOperations + .absolutePath(context.cmd.targetDeltaLog.dataPath.toString, f.path) + .toString, f)).toMap + + // When the target table is empty, and the optimizer optimized away the join entirely + // numSourceRows will be incorrectly 0. + // We need to scan the source table once to get the correct + // metric here. + if (context.cmd.metrics("numSourceRows").value == 0 && + (dataSkippedFiles.isEmpty || dataSkippedTargetDF.take(1).isEmpty)) { + val numSourceRows = sourceDF.count() + context.cmd.metrics("numSourceRows").set(numSourceRows) + } + + // Update metrics + context.cmd.metrics("numTargetFilesBeforeSkipping") += context.deltaTxn.snapshot.numOfFiles + context.cmd.metrics("numTargetBytesBeforeSkipping") += context.deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + context.cmd.metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + context.cmd.metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + context.cmd.metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + val (removedBytes, removedPartitions) = + totalBytesAndDistinctPartitionValues(touchedAddFiles.values.toSeq) + context.cmd.metrics("numTargetFilesRemoved") += touchedAddFiles.size + context.cmd.metrics("numTargetBytesRemoved") += removedBytes + context.cmd.metrics("numTargetPartitionsRemovedFrom") += removedPartitions + + collectTouchedFiles.map(kv => (kv._1, (kv._2._1, touchedAddFiles(kv._1)))) + } + + + /** + * Modify original data frame to insert + * [[GpuDeltaParquetFileFormatUtils.METADATA_ROW_IDX_COL]]. + */ + private def addRowIndexMetaColumn(baseDF: DataFrame): DataFrame = { + val rowIdxAttr = AttributeReference( + METADATA_ROW_IDX_COL, + METADATA_ROW_IDX_FIELD.dataType, + METADATA_ROW_IDX_FIELD.nullable)() + + val newPlan = baseDF.queryExecution.analyzed.transformUp { + case r@LogicalRelation(fs: HadoopFsRelation, _, _, _) => + val newSchema = StructType(fs.dataSchema.fields).add(METADATA_ROW_IDX_FIELD) + + // This is required to ensure that row index is correctly calculated. + val newFileFormat = fs.fileFormat.asInstanceOf[DeltaParquetFileFormat] + .copy(isSplittable = false, disablePushDowns = true) + + val newFs = fs.copy(dataSchema = newSchema, fileFormat = newFileFormat)(context.spark) + + val newOutput = r.output :+ rowIdxAttr + r.copy(relation = newFs, output = newOutput) + case p@Project(projectList, _) => + val newProjectList = projectList :+ rowIdxAttr + p.copy(projectList = newProjectList) + } + + Dataset.ofRows(context.spark, newPlan) + } + + /** + * The result is scanning target table with touched files, and added an extra + * [[METADATA_ROW_DEL_COL]] to indicate whether filtered by joining with source table in first + * step. + */ + private def getTouchedTargetDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]) + : DataFrame = { + // Generate a new target dataframe that has same output attributes exprIds as the target plan. + // This allows us to apply the existing resolved update/insert expressions. + val baseTargetDF = buildTargetDFWithFiles(touchedFiles.values.map(_._2).toSeq) + + val newPlan = { + val rowDelAttr = AttributeReference( + METADATA_ROW_DEL_COL, + METADATA_ROW_DEL_FIELD.dataType, + METADATA_ROW_DEL_FIELD.nullable)() + + baseTargetDF.queryExecution.analyzed.transformUp { + case r@LogicalRelation(fs: HadoopFsRelation, _, _, _) => + val newSchema = StructType(fs.dataSchema.fields).add(METADATA_ROW_DEL_FIELD) + + // This is required to ensure that row index is correctly calculated. + val newFileFormat = { + val oldFormat = fs.fileFormat.asInstanceOf[DeltaParquetFileFormat] + val dvs = touchedFiles.map(kv => (new URI(kv._1), + DeletionVectorDescriptorWithFilterType(toDeletionVector(kv._2._1), + RowIndexFilterType.UNKNOWN))) + val broadcastDVs = context.spark.sparkContext.broadcast(dvs) + + oldFormat.copy(isSplittable = false, + broadcastDvMap = Some(broadcastDVs), + disablePushDowns = true) + } + + val newFs = fs.copy(dataSchema = newSchema, fileFormat = newFileFormat)(context.spark) + + val newOutput = r.output :+ rowDelAttr + r.copy(relation = newFs, output = newOutput) + case p@Project(projectList, _) => + val newProjectList = projectList :+ rowDelAttr + p.copy(projectList = newProjectList) + } + } + + val df = Dataset.ofRows(context.spark, newPlan) + .withColumn(TARGET_ROW_PRESENT_COL, lit(true)) + + df + } + + /** + * Generate a plan by calculating modified rows. It's computed by joining source and target + * tables, where target table has been filtered by (`__metadata_file_name`, + * `__metadata_row_idx`) pairs collected in first step. + * + * Schema of `modifiedDF`: + * + * targetSchema + ROW_DROPPED_COL + TARGET_ROW_PRESENT_COL + + * SOURCE_ROW_PRESENT_COL + INCR_METRICS_COL + * INCR_METRICS_COL + * + * It consists of several parts: + * + * 1. Unmatched source rows which are inserted + * 2. Unmatched source rows which are deleted + * 3. Target rows which are updated + * 4. Target rows which are deleted + */ + private def getModifiedDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]): DataFrame = { + val sourceDF = this.sourceDF + .withColumn(SOURCE_ROW_PRESENT_COL, new Column(incrSourceRowCountExpr)) + + val targetDF = getTouchedTargetDF(touchedFiles) + + val joinedDF = { + val joinType = if (hasNoInserts && + context.spark.conf.get(DeltaSQLConf.MERGE_MATCHED_ONLY_ENABLED)) { + "inner" + } else { + "leftOuter" + } + val matchedTargetDF = targetDF.filter(METADATA_ROW_DEL_COL) + .drop(METADATA_ROW_DEL_COL) + + sourceDF.join(matchedTargetDF, new Column(context.cmd.condition), joinType) + } + + val modifiedRowsSchema = context.deltaTxn.metadata.schema + .add(ROW_DROPPED_FIELD) + .add(TARGET_ROW_PRESENT_FIELD.copy(nullable = true)) + .add(SOURCE_ROW_PRESENT_FIELD.copy(nullable = true)) + .add(INCR_METRICS_FIELD) + + // Here we generate a case when statement to handle all cases: + // CASE + // WHEN + // CASE WHEN + // + // WHEN + // + // ELSE + // + // WHEN + // CASE WHEN + // + // WHEN + // + // ELSE + // + // END + + val notMatchedConditions = context.cmd.notMatchedClauses.map(clauseCondition) + val notMatchedExpr = { + val deletedNotMatchedRow = { + targetOutputCols :+ + Literal.TrueLiteral :+ + Literal.FalseLiteral :+ + Literal(null) :+ + Literal.TrueLiteral + } + if (context.cmd.notMatchedClauses.isEmpty) { + // If there no `WHEN NOT MATCHED` clause, we should just delete not matched row + deletedNotMatchedRow + } else { + val notMatchedOutputs = context.cmd.notMatchedClauses.map(clauseOutput) + modifiedRowsSchema.zipWithIndex.map { + case (_, idx) => + CaseWhen(notMatchedConditions.zip(notMatchedOutputs.map(_(idx))), + deletedNotMatchedRow(idx)) + } + } + } + + val matchedConditions = context.cmd.matchedClauses.map(clauseCondition) + val matchedOutputs = context.cmd.matchedClauses.map(clauseOutput) + val matchedExprs = { + val notMatchedRow = { + targetOutputCols :+ + Literal.FalseLiteral :+ + Literal.TrueLiteral :+ + Literal(null) :+ + Literal.TrueLiteral + } + if (context.cmd.matchedClauses.isEmpty) { + // If there is not matched clause, this is insert only, we should delete this row. + notMatchedRow + } else { + modifiedRowsSchema.zipWithIndex.map { + case (_, idx) => + CaseWhen(matchedConditions.zip(matchedOutputs.map(_(idx))), + notMatchedRow(idx)) + } + } + } + + val sourceRowHasNoMatch = col(TARGET_ROW_PRESENT_COL).isNull.expr + + val modifiedCols = modifiedRowsSchema.zipWithIndex.map { case (col, idx) => + val caseWhen = CaseWhen( + Seq(sourceRowHasNoMatch -> notMatchedExpr(idx)), + matchedExprs(idx)) + new Column(Alias(caseWhen, col.name)()) + } + + val modifiedDF = { + + // Make this a udf to avoid catalyst to be too aggressive to even remove the join! + val noopRowDroppedCol = udf(new GpuDeltaNoopUDF()).apply(!col(ROW_DROPPED_COL)) + + val modifiedDF = joinedDF.select(modifiedCols: _*) + // This will not filter anything since they always return true, but we need to avoid + // catalyst from optimizing these udf + .filter(noopRowDroppedCol && col(INCR_METRICS_COL)) + .drop(ROW_DROPPED_COL, INCR_METRICS_COL, TARGET_ROW_PRESENT_COL, SOURCE_ROW_PRESENT_COL) + + repartitionIfNeeded(modifiedDF) + } + + modifiedDF + } + + private def getUnmodifiedDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]): DataFrame = { + getTouchedTargetDF(touchedFiles) + .filter(!col(METADATA_ROW_DEL_COL)) + .drop(TARGET_ROW_PRESENT_COL, METADATA_ROW_DEL_COL) + } +} + + +object MergeExecutor { + + /** + * Spark UI will track all normal accumulators along with Spark tasks to show them on Web UI. + * However, the accumulator used by `MergeIntoCommand` can store a very large value since it + * tracks all files that need to be rewritten. We should ask Spark UI to not remember it, + * otherwise, the UI data may consume lots of memory. Hence, we use the prefix `internal.metrics.` + * to make this accumulator become an internal accumulator, so that it will not be tracked by + * Spark UI. + */ + val TOUCHED_FILES_ACCUM_NAME = "internal.metrics.MergeIntoDelta.touchedFiles" + + val ROW_ID_COL = "_row_id_" + val FILE_PATH_COL: String = GpuDeltaParquetFileFormatUtils.FILE_PATH_COL + val SOURCE_ROW_PRESENT_COL: String = "_source_row_present_" + val SOURCE_ROW_PRESENT_FIELD: StructField = StructField(SOURCE_ROW_PRESENT_COL, BooleanType, + nullable = false) + val TARGET_ROW_PRESENT_COL: String = "_target_row_present_" + val TARGET_ROW_PRESENT_FIELD: StructField = StructField(TARGET_ROW_PRESENT_COL, BooleanType, + nullable = false) + val ROW_DROPPED_COL: String = GpuDeltaMergeConstants.ROW_DROPPED_COL + val ROW_DROPPED_FIELD: StructField = StructField(ROW_DROPPED_COL, BooleanType, nullable = false) + val INCR_METRICS_COL: String = "_incr_metrics_" + val INCR_METRICS_FIELD: StructField = StructField(INCR_METRICS_COL, BooleanType, nullable = false) + val INCR_ROW_COUNT_COL: String = "_incr_row_count_" + + // Some Delta versions use Literal(null) which translates to a literal of NullType instead + // of the Literal(null, StringType) which is needed, so using a fixed version here + // rather than the version from Delta Lake. + val CDC_TYPE_NOT_CDC_LITERAL: Literal = Literal(null, StringType) + + def toDeletionVector(bitmap: Roaring64Bitmap): DeletionVectorDescriptor = { + DeletionVectorDescriptor.inlineInLog(RoaringBitmapWrapper(bitmap).serializeToBytes(), + bitmap.getLongCardinality) + } + + /** Count the number of distinct partition values among the AddFiles in the given set. */ + def totalBytesAndDistinctPartitionValues(files: Seq[FileAction]): (Long, Int) = { + val distinctValues = new mutable.HashSet[Map[String, String]]() + var bytes = 0L + val iter = files.collect { case a: AddFile => a }.iterator + while (iter.hasNext) { + val file = iter.next() + distinctValues += file.partitionValues + bytes += file.size + } + // If the only distinct value map is an empty map, then it must be an unpartitioned table. + // Return 0 in that case. + val numDistinctValues = + if (distinctValues.size == 1 && distinctValues.head.isEmpty) 0 else distinctValues.size + (bytes, numDistinctValues) + } +} \ No newline at end of file diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala new file mode 100644 index 00000000000..c7d24cd81bc --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala @@ -0,0 +1,1189 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from MergeIntoCommand.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.DeltaOperations.MergePredicate +import com.databricks.sql.transaction.tahoe.actions.{AddCDCFile, AddFile, FileAction} +import com.databricks.sql.transaction.tahoe.commands.DeltaCommand +import com.databricks.sql.transaction.tahoe.schema.ImplicitMetadataOperation +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.databricks.sql.transaction.tahoe.util.{AnalysisHelper, SetAccumulator} +import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import com.nvidia.spark.rapids.{BaseExprMeta, GpuOverrides, RapidsConf} +import com.nvidia.spark.rapids.delta._ + +import org.apache.spark.SparkContext +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, BasePredicate, Expression, Literal, NamedExpression, PredicateHelper, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate +import org.apache.spark.sql.catalyst.plans.logical.{DeltaMergeIntoClause, DeltaMergeIntoMatchedClause, DeltaMergeIntoMatchedDeleteClause, DeltaMergeIntoMatchedUpdateClause, DeltaMergeIntoNotMatchedBySourceClause, DeltaMergeIntoNotMatchedClause, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DataTypes, LongType, StringType, StructType} +case class GpuMergeDataSizes( + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + rows: Option[Long] = None, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + files: Option[Long] = None, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + bytes: Option[Long] = None, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + partitions: Option[Long] = None) + +/** + * Represents the state of a single merge clause: + * - merge clause's (optional) predicate + * - action type (insert, update, delete) + * - action's expressions + */ +case class GpuMergeClauseStats( + condition: Option[String], + actionType: String, + actionExpr: Seq[String]) + +object GpuMergeClauseStats { + def apply(mergeClause: DeltaMergeIntoClause): GpuMergeClauseStats = { + GpuMergeClauseStats( + condition = mergeClause.condition.map(_.sql), + mergeClause.clauseType.toLowerCase(), + actionExpr = mergeClause.actions.map(_.sql)) + } +} + +/** State for a GPU merge operation */ +case class GpuMergeStats( + // Merge condition expression + conditionExpr: String, + + // Expressions used in old MERGE stats, now always Null + updateConditionExpr: String, + updateExprs: Seq[String], + insertConditionExpr: String, + insertExprs: Seq[String], + deleteConditionExpr: String, + + // Newer expressions used in MERGE with any number of MATCHED/NOT MATCHED + matchedStats: Seq[GpuMergeClauseStats], + notMatchedStats: Seq[GpuMergeClauseStats], + + // Data sizes of source and target at different stages of processing + source: GpuMergeDataSizes, + targetBeforeSkipping: GpuMergeDataSizes, + targetAfterSkipping: GpuMergeDataSizes, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + sourceRowsInSecondScan: Option[Long], + + // Data change sizes + targetFilesRemoved: Long, + targetFilesAdded: Long, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + targetChangeFilesAdded: Option[Long], + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + targetChangeFileBytes: Option[Long], + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + targetBytesRemoved: Option[Long], + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + targetBytesAdded: Option[Long], + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + targetPartitionsRemovedFrom: Option[Long], + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + targetPartitionsAddedTo: Option[Long], + targetRowsCopied: Long, + targetRowsUpdated: Long, + targetRowsInserted: Long, + targetRowsDeleted: Long +) + +object GpuMergeStats { + + def fromMergeSQLMetrics( + metrics: Map[String, SQLMetric], + condition: Expression, + matchedClauses: Seq[DeltaMergeIntoMatchedClause], + notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], + isPartitioned: Boolean): GpuMergeStats = { + + def metricValueIfPartitioned(metricName: String): Option[Long] = { + if (isPartitioned) Some(metrics(metricName).value) else None + } + + GpuMergeStats( + // Merge condition expression + conditionExpr = condition.sql, + + // Newer expressions used in MERGE with any number of MATCHED/NOT MATCHED + matchedStats = matchedClauses.map(GpuMergeClauseStats(_)), + notMatchedStats = notMatchedClauses.map(GpuMergeClauseStats(_)), + + // Data sizes of source and target at different stages of processing + source = GpuMergeDataSizes(rows = Some(metrics("numSourceRows").value)), + targetBeforeSkipping = + GpuMergeDataSizes( + files = Some(metrics("numTargetFilesBeforeSkipping").value), + bytes = Some(metrics("numTargetBytesBeforeSkipping").value)), + targetAfterSkipping = + GpuMergeDataSizes( + files = Some(metrics("numTargetFilesAfterSkipping").value), + bytes = Some(metrics("numTargetBytesAfterSkipping").value), + partitions = metricValueIfPartitioned("numTargetPartitionsAfterSkipping")), + sourceRowsInSecondScan = + metrics.get("numSourceRowsInSecondScan").map(_.value).filter(_ >= 0), + + // Data change sizes + targetFilesAdded = metrics("numTargetFilesAdded").value, + targetChangeFilesAdded = metrics.get("numTargetChangeFilesAdded").map(_.value), + targetChangeFileBytes = metrics.get("numTargetChangeFileBytes").map(_.value), + targetFilesRemoved = metrics("numTargetFilesRemoved").value, + targetBytesAdded = Some(metrics("numTargetBytesAdded").value), + targetBytesRemoved = Some(metrics("numTargetBytesRemoved").value), + targetPartitionsRemovedFrom = metricValueIfPartitioned("numTargetPartitionsRemovedFrom"), + targetPartitionsAddedTo = metricValueIfPartitioned("numTargetPartitionsAddedTo"), + targetRowsCopied = metrics("numTargetRowsCopied").value, + targetRowsUpdated = metrics("numTargetRowsUpdated").value, + targetRowsInserted = metrics("numTargetRowsInserted").value, + targetRowsDeleted = metrics("numTargetRowsDeleted").value, + + // Deprecated fields + updateConditionExpr = null, + updateExprs = null, + insertConditionExpr = null, + insertExprs = null, + deleteConditionExpr = null) + } +} + +/** + * GPU version of Delta Lake's MergeIntoCommand. + * + * Performs a merge of a source query/table into a Delta table. + * + * Issues an error message when the ON search_condition of the MERGE statement can match + * a single row from the target table with multiple rows of the source table-reference. + * + * Algorithm: + * + * Phase 1: Find the input files in target that are touched by the rows that satisfy + * the condition and verify that no two source rows match with the same target row. + * This is implemented as an inner-join using the given condition. See [[findTouchedFiles]] + * for more details. + * + * Phase 2: Read the touched files again and write new files with updated and/or inserted rows. + * + * Phase 3: Use the Delta protocol to atomically remove the touched files and add the new files. + * + * @param source Source data to merge from + * @param target Target table to merge into + * @param gpuDeltaLog Delta log to use + * @param condition Condition for a source row to match with a target row + * @param matchedClauses All info related to matched clauses. + * @param notMatchedClauses All info related to not matched clause. + * @param migratedSchema The final schema of the target - may be changed by schema evolution. + */ +case class GpuMergeIntoCommand( + @transient source: LogicalPlan, + @transient target: LogicalPlan, + @transient gpuDeltaLog: GpuDeltaLog, + condition: Expression, + matchedClauses: Seq[DeltaMergeIntoMatchedClause], + notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], + notMatchedBySourceClauses: Seq[DeltaMergeIntoNotMatchedBySourceClause], + migratedSchema: Option[StructType])( + @transient val rapidsConf: RapidsConf) + extends LeafRunnableCommand + with DeltaCommand with PredicateHelper with AnalysisHelper with ImplicitMetadataOperation { + + import GpuMergeIntoCommand._ + + import SQLMetrics._ + import com.databricks.sql.transaction.tahoe.commands.cdc.CDCReader._ + + override val otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) + + override val canMergeSchema: Boolean = conf.getConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE) + override val canOverwriteSchema: Boolean = false + + override val output: Seq[Attribute] = Seq( + AttributeReference("num_affected_rows", LongType)(), + AttributeReference("num_updated_rows", LongType)(), + AttributeReference("num_deleted_rows", LongType)(), + AttributeReference("num_inserted_rows", LongType)()) + + @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() + @transient private lazy val targetDeltaLog: DeltaLog = gpuDeltaLog.deltaLog + /** + * Map to get target output attributes by name. + * The case sensitivity of the map is set accordingly to Spark configuration. + */ + @transient private lazy val targetOutputAttributesMap: Map[String, Attribute] = { + val attrMap: Map[String, Attribute] = target + .outputSet.view + .map(attr => attr.name -> attr).toMap + if (conf.caseSensitiveAnalysis) { + attrMap + } else { + CaseInsensitiveMap(attrMap) + } + } + + /** Whether this merge statement has only a single insert (NOT MATCHED) clause. */ + private def isSingleInsertOnly: Boolean = matchedClauses.isEmpty && notMatchedClauses.length == 1 + /** Whether this merge statement has only MATCHED clauses. */ + private def isMatchedOnly: Boolean = notMatchedClauses.isEmpty && matchedClauses.nonEmpty + + // We over-count numTargetRowsDeleted when there are multiple matches; + // this is the amount of the overcount, so we can subtract it to get a correct final metric. + private var multipleMatchDeleteOnlyOvercount: Option[Long] = None + + override lazy val metrics = Map[String, SQLMetric]( + "numSourceRows" -> createMetric(sc, "number of source rows"), + "numSourceRowsInSecondScan" -> + createMetric(sc, "number of source rows (during repeated scan)"), + "numTargetRowsCopied" -> createMetric(sc, "number of target rows rewritten unmodified"), + "numTargetRowsInserted" -> createMetric(sc, "number of inserted rows"), + "numTargetRowsUpdated" -> createMetric(sc, "number of updated rows"), + "numTargetRowsDeleted" -> createMetric(sc, "number of deleted rows"), + "numTargetFilesBeforeSkipping" -> createMetric(sc, "number of target files before skipping"), + "numTargetFilesAfterSkipping" -> createMetric(sc, "number of target files after skipping"), + "numTargetFilesRemoved" -> createMetric(sc, "number of files removed to target"), + "numTargetFilesAdded" -> createMetric(sc, "number of files added to target"), + "numTargetChangeFilesAdded" -> + createMetric(sc, "number of change data capture files generated"), + "numTargetChangeFileBytes" -> + createMetric(sc, "total size of change data capture files generated"), + "numTargetBytesBeforeSkipping" -> createMetric(sc, "number of target bytes before skipping"), + "numTargetBytesAfterSkipping" -> createMetric(sc, "number of target bytes after skipping"), + "numTargetBytesRemoved" -> createMetric(sc, "number of target bytes removed"), + "numTargetBytesAdded" -> createMetric(sc, "number of target bytes added"), + "numTargetPartitionsAfterSkipping" -> + createMetric(sc, "number of target partitions after skipping"), + "numTargetPartitionsRemovedFrom" -> + createMetric(sc, "number of target partitions from which files were removed"), + "numTargetPartitionsAddedTo" -> + createMetric(sc, "number of target partitions to which files were added"), + "executionTimeMs" -> + createMetric(sc, "time taken to execute the entire operation"), + "scanTimeMs" -> + createMetric(sc, "time taken to scan the files for matches"), + "rewriteTimeMs" -> + createMetric(sc, "time taken to rewrite the matched files")) + + override def run(spark: SparkSession): Seq[Row] = { + recordDeltaOperation(targetDeltaLog, "delta.dml.merge") { + val startTime = System.nanoTime() + gpuDeltaLog.withNewTransaction { deltaTxn => + if (target.schema.size != deltaTxn.metadata.schema.size) { + throw DeltaErrors.schemaChangedSinceAnalysis( + atAnalysis = target.schema, latestSchema = deltaTxn.metadata.schema) + } + + if (canMergeSchema) { + updateMetadata( + spark, deltaTxn, migratedSchema.getOrElse(target.schema), + deltaTxn.metadata.partitionColumns, deltaTxn.metadata.configuration, + isOverwriteMode = false, rearrangeOnly = false) + } + + val deltaActions = { + if (isSingleInsertOnly && spark.conf.get(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED)) { + writeInsertsOnlyWhenNoMatchedClauses(spark, deltaTxn) + } else { + val filesToRewrite = findTouchedFiles(spark, deltaTxn) + val newWrittenFiles = withStatusCode("DELTA", "Writing merged data") { + writeAllChanges(spark, deltaTxn, filesToRewrite) + } + filesToRewrite.map(_.remove) ++ newWrittenFiles + } + } + + // Metrics should be recorded before commit (where they are written to delta logs). + metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) + deltaTxn.registerSQLMetrics(spark, metrics) + + // This is a best-effort sanity check. + if (metrics("numSourceRowsInSecondScan").value >= 0 && + metrics("numSourceRows").value != metrics("numSourceRowsInSecondScan").value) { + log.warn(s"Merge source has ${metrics("numSourceRows").value} rows in initial scan but " + + s"${metrics("numSourceRowsInSecondScan").value} rows in second scan") + if (conf.getConf(DeltaSQLConf.MERGE_FAIL_IF_SOURCE_CHANGED)) { + throw DeltaErrors.sourceNotDeterministicInMergeException(spark) + } + } + + deltaTxn.commit( + deltaActions, + DeltaOperations.Merge( + Option(condition), + matchedClauses.map(DeltaOperations.MergePredicate(_)), + notMatchedClauses.map(DeltaOperations.MergePredicate(_)), + // We do not support notMatchedBySourcePredicates yet and fall back to CPU + // See https://github.com/NVIDIA/spark-rapids/issues/8415 + notMatchedBySourcePredicates = Seq.empty[MergePredicate] + )) + + // Record metrics + val stats = GpuMergeStats.fromMergeSQLMetrics( + metrics, condition, matchedClauses, notMatchedClauses, + deltaTxn.metadata.partitionColumns.nonEmpty) + recordDeltaEvent(targetDeltaLog, "delta.dml.merge.stats", data = stats) + + } + spark.sharedState.cacheManager.recacheByPlan(spark, target) + } + // This is needed to make the SQL metrics visible in the Spark UI. Also this needs + // to be outside the recordMergeOperation because this method will update some metric. + val executionId = spark.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(spark.sparkContext, executionId, metrics.values.toSeq) + Seq(Row(metrics("numTargetRowsUpdated").value + metrics("numTargetRowsDeleted").value + + metrics("numTargetRowsInserted").value, metrics("numTargetRowsUpdated").value, + metrics("numTargetRowsDeleted").value, metrics("numTargetRowsInserted").value)) + } + + /** + * Find the target table files that contain the rows that satisfy the merge condition. This is + * implemented as an inner-join between the source query/table and the target table using + * the merge condition. + */ + private def findTouchedFiles( + spark: SparkSession, + deltaTxn: OptimisticTransaction + ): Seq[AddFile] = recordMergeOperation(sqlMetricName = "scanTimeMs") { + + // Accumulator to collect all the distinct touched files + val touchedFilesAccum = new SetAccumulator[String]() + spark.sparkContext.register(touchedFilesAccum, TOUCHED_FILES_ACCUM_NAME) + + // UDFs to records touched files names and add them to the accumulator + val recordTouchedFileName = udf(new GpuDeltaRecordTouchedFileNameUDF(touchedFilesAccum)) + .asNondeterministic() + + // Skip data based on the merge condition + val targetOnlyPredicates = + splitConjunctivePredicates(condition).filter(_.references.subsetOf(target.outputSet)) + val dataSkippedFiles = deltaTxn.filterFiles(targetOnlyPredicates) + + // UDF to increment metrics + val incrSourceRowCountExpr = makeMetricUpdateUDF("numSourceRows") + val sourceDF = Dataset.ofRows(spark, source) + .filter(new Column(incrSourceRowCountExpr)) + + // Apply inner join to between source and target using the merge condition to find matches + // In addition, we attach two columns + // - a monotonically increasing row id for target rows to later identify whether the same + // target row is modified by multiple user or not + // - the target file name the row is from to later identify the files touched by matched rows + val targetDF = Dataset.ofRows(spark, buildTargetPlanWithFiles(deltaTxn, dataSkippedFiles)) + .withColumn(ROW_ID_COL, monotonically_increasing_id()) + .withColumn(FILE_NAME_COL, input_file_name()) + val joinToFindTouchedFiles = sourceDF.join(targetDF, new Column(condition), "inner") + + // Process the matches from the inner join to record touched files and find multiple matches + val collectTouchedFiles = joinToFindTouchedFiles + .select(col(ROW_ID_COL), recordTouchedFileName(col(FILE_NAME_COL)).as("one")) + + // Calculate frequency of matches per source row + val matchedRowCounts = collectTouchedFiles.groupBy(ROW_ID_COL).agg(sum("one").as("count")) + + // Get multiple matches and simultaneously collect (using touchedFilesAccum) the file names + // multipleMatchCount = # of target rows with more than 1 matching source row (duplicate match) + // multipleMatchSum = total # of duplicate matched rows + import spark.implicits._ + val (multipleMatchCount, multipleMatchSum) = matchedRowCounts + .filter("count > 1") + .select(coalesce(count("*"), lit(0)), coalesce(sum("count"), lit(0))) + .as[(Long, Long)] + .collect() + .head + + val hasMultipleMatches = multipleMatchCount > 0 + + // Throw error if multiple matches are ambiguous or cannot be computed correctly. + val canBeComputedUnambiguously = { + // Multiple matches are not ambiguous when there is only one unconditional delete as + // all the matched row pairs in the 2nd join in `writeAllChanges` will get deleted. + val isUnconditionalDelete = matchedClauses.headOption match { + case Some(DeltaMergeIntoMatchedDeleteClause(None)) => true + case _ => false + } + matchedClauses.size == 1 && isUnconditionalDelete + } + + if (hasMultipleMatches && !canBeComputedUnambiguously) { + throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException(spark) + } + + if (hasMultipleMatches) { + // This is only allowed for delete-only queries. + // This query will count the duplicates for numTargetRowsDeleted in Job 2, + // because we count matches after the join and not just the target rows. + // We have to compensate for this by subtracting the duplicates later, + // so we need to record them here. + val duplicateCount = multipleMatchSum - multipleMatchCount + multipleMatchDeleteOnlyOvercount = Some(duplicateCount) + } + + // Get the AddFiles using the touched file names. + val touchedFileNames = touchedFilesAccum.value.iterator().asScala.toSeq + logTrace(s"findTouchedFiles: matched files:\n\t${touchedFileNames.mkString("\n\t")}") + + val nameToAddFileMap = generateCandidateFileMap(targetDeltaLog.dataPath, dataSkippedFiles) + val touchedAddFiles = touchedFileNames.map(f => + getTouchedFile(targetDeltaLog.dataPath, f, nameToAddFileMap)) + + // When the target table is empty, and the optimizer optimized away the join entirely + // numSourceRows will be incorrectly 0. We need to scan the source table once to get the correct + // metric here. + if (metrics("numSourceRows").value == 0 && + (dataSkippedFiles.isEmpty || targetDF.take(1).isEmpty)) { + val numSourceRows = sourceDF.count() + metrics("numSourceRows").set(numSourceRows) + } + + // Update metrics + metrics("numTargetFilesBeforeSkipping") += deltaTxn.snapshot.numOfFiles + metrics("numTargetBytesBeforeSkipping") += deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + val (removedBytes, removedPartitions) = totalBytesAndDistinctPartitionValues(touchedAddFiles) + metrics("numTargetFilesRemoved") += touchedAddFiles.size + metrics("numTargetBytesRemoved") += removedBytes + metrics("numTargetPartitionsRemovedFrom") += removedPartitions + touchedAddFiles + } + + /** + * This is an optimization of the case when there is no update clause for the merge. + * We perform an left anti join on the source data to find the rows to be inserted. + * + * This will currently only optimize for the case when there is a _single_ notMatchedClause. + */ + private def writeInsertsOnlyWhenNoMatchedClauses( + spark: SparkSession, + deltaTxn: OptimisticTransaction + ): Seq[FileAction] = recordMergeOperation(sqlMetricName = "rewriteTimeMs") { + + // UDFs to update metrics + val incrSourceRowCountExpr = makeMetricUpdateUDF("numSourceRows") + val incrInsertedCountExpr = makeMetricUpdateUDF("numTargetRowsInserted") + + val outputColNames = getTargetOutputCols(deltaTxn).map(_.name) + // we use head here since we know there is only a single notMatchedClause + val outputExprs = notMatchedClauses.head.resolvedActions.map(_.expr) + val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => + new Column(Alias(expr, name)()) + } + + // source DataFrame + val sourceDF = Dataset.ofRows(spark, source) + .filter(new Column(incrSourceRowCountExpr)) + .filter(new Column(notMatchedClauses.head.condition.getOrElse(Literal.TrueLiteral))) + + // Skip data based on the merge condition + val conjunctivePredicates = splitConjunctivePredicates(condition) + val targetOnlyPredicates = + conjunctivePredicates.filter(_.references.subsetOf(target.outputSet)) + val dataSkippedFiles = deltaTxn.filterFiles(targetOnlyPredicates) + + // target DataFrame + val targetDF = Dataset.ofRows( + spark, buildTargetPlanWithFiles(deltaTxn, dataSkippedFiles)) + + val insertDf = sourceDF.join(targetDF, new Column(condition), "leftanti") + .select(outputCols: _*) + .filter(new Column(incrInsertedCountExpr)) + + val newFiles = deltaTxn + .writeFiles(repartitionIfNeeded(spark, insertDf, deltaTxn.metadata.partitionColumns)) + + // Update metrics + metrics("numTargetFilesBeforeSkipping") += deltaTxn.snapshot.numOfFiles + metrics("numTargetBytesBeforeSkipping") += deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + metrics("numTargetFilesRemoved") += 0 + metrics("numTargetBytesRemoved") += 0 + metrics("numTargetPartitionsRemovedFrom") += 0 + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + metrics("numTargetBytesAdded") += addedBytes + metrics("numTargetPartitionsAddedTo") += addedPartitions + newFiles + } + + /** + * Write new files by reading the touched files and updating/inserting data using the source + * query/table. This is implemented using a full|right-outer-join using the merge condition. + * + * Note that unlike the insert-only code paths with just one control column INCR_ROW_COUNT_COL, + * this method has two additional control columns ROW_DROPPED_COL for dropping deleted rows and + * CDC_TYPE_COL_NAME used for handling CDC when enabled. + */ + private def writeAllChanges( + spark: SparkSession, + deltaTxn: OptimisticTransaction, + filesToRewrite: Seq[AddFile] + ): Seq[FileAction] = recordMergeOperation(sqlMetricName = "rewriteTimeMs") { + import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} + + val cdcEnabled = DeltaConfigs.CHANGE_DATA_FEED.fromMetaData(deltaTxn.metadata) + + var targetOutputCols = getTargetOutputCols(deltaTxn) + var outputRowSchema = deltaTxn.metadata.schema + + // When we have duplicate matches (only allowed when the whenMatchedCondition is a delete with + // no match condition) we will incorrectly generate duplicate CDC rows. + // Duplicate matches can be due to: + // - Duplicate rows in the source w.r.t. the merge condition + // - A target-only or source-only merge condition, which essentially turns our join into a cross + // join with the target/source satisfiying the merge condition. + // These duplicate matches are dropped from the main data output since this is a delete + // operation, but the duplicate CDC rows are not removed by default. + // See https://github.com/delta-io/delta/issues/1274 + + // We address this specific scenario by adding row ids to the target before performing our join. + // There should only be one CDC delete row per target row so we can use these row ids to dedupe + // the duplicate CDC delete rows. + + // We also need to address the scenario when there are duplicate matches with delete and we + // insert duplicate rows. Here we need to additionally add row ids to the source before the + // join to avoid dropping these valid duplicate inserted rows and their corresponding cdc rows. + + // When there is an insert clause, we set SOURCE_ROW_ID_COL=null for all delete rows because we + // need to drop the duplicate matches. + val isDeleteWithDuplicateMatchesAndCdc = multipleMatchDeleteOnlyOvercount.nonEmpty && cdcEnabled + + // Generate a new logical plan that has same output attributes exprIds as the target plan. + // This allows us to apply the existing resolved update/insert expressions. + val newTarget = buildTargetPlanWithFiles(deltaTxn, filesToRewrite) + val joinType = if (isMatchedOnly && + spark.conf.get(DeltaSQLConf.MERGE_MATCHED_ONLY_ENABLED)) { + "rightOuter" + } else { + "fullOuter" + } + + logDebug(s"""writeAllChanges using $joinType join: + | source.output: ${source.outputSet} + | target.output: ${target.outputSet} + | condition: $condition + | newTarget.output: ${newTarget.outputSet} + """.stripMargin) + + // UDFs to update metrics + // Make UDFs that appear in the custom join processor node deterministic, as they always + // return true and update a metric. Catalyst precludes non-deterministic UDFs that are not + // allowed outside a very specific set of Catalyst nodes (Project, Filter, Window, Aggregate). + val incrSourceRowCountExpr = makeMetricUpdateUDF("numSourceRowsInSecondScan") + val incrUpdatedCountExpr = makeMetricUpdateUDF("numTargetRowsUpdated", deterministic = true) + val incrInsertedCountExpr = makeMetricUpdateUDF("numTargetRowsInserted", deterministic = true) + val incrNoopCountExpr = makeMetricUpdateUDF("numTargetRowsCopied", deterministic = true) + val incrDeletedCountExpr = makeMetricUpdateUDF("numTargetRowsDeleted", deterministic = true) + + // Apply an outer join to find both, matches and non-matches. We are adding two boolean fields + // with value `true`, one to each side of the join. Whether this field is null or not after + // the outer join, will allow us to identify whether the resultant joined row was a + // matched inner result or an unmatched result with null on one side. + // We add row IDs to the targetDF if we have a delete-when-matched clause with duplicate + // matches and CDC is enabled, and additionally add row IDs to the source if we also have an + // insert clause. See above at isDeleteWithDuplicateMatchesAndCdc definition for more details. + var sourceDF = Dataset.ofRows(spark, source) + .withColumn(SOURCE_ROW_PRESENT_COL, new Column(incrSourceRowCountExpr)) + var targetDF = Dataset.ofRows(spark, newTarget) + .withColumn(TARGET_ROW_PRESENT_COL, lit(true)) + if (isDeleteWithDuplicateMatchesAndCdc) { + targetDF = targetDF.withColumn(TARGET_ROW_ID_COL, monotonically_increasing_id()) + if (notMatchedClauses.nonEmpty) { // insert clause + sourceDF = sourceDF.withColumn(SOURCE_ROW_ID_COL, monotonically_increasing_id()) + } + } + val joinedDF = sourceDF.join(targetDF, new Column(condition), joinType) + val joinedPlan = joinedDF.queryExecution.analyzed + + def resolveOnJoinedPlan(exprs: Seq[Expression]): Seq[Expression] = { + tryResolveReferencesForExpressions(spark, exprs, joinedPlan) + } + + // ==== Generate the expressions to process full-outer join output and generate target rows ==== + // If there are N columns in the target table, there will be N + 3 columns after processing + // - N columns for target table + // - ROW_DROPPED_COL to define whether the generated row should dropped or written + // - INCR_ROW_COUNT_COL containing a UDF to update the output row row counter + // - CDC_TYPE_COLUMN_NAME containing the type of change being performed in a particular row + + // To generate these N + 3 columns, we will generate N + 3 expressions and apply them to the + // rows in the joinedDF. The CDC column will be either used for CDC generation or dropped before + // performing the final write, and the other two will always be dropped after executing the + // metrics UDF and filtering on ROW_DROPPED_COL. + + // We produce rows for both the main table data (with CDC_TYPE_COLUMN_NAME = CDC_TYPE_NOT_CDC), + // and rows for the CDC data which will be output to CDCReader.CDC_LOCATION. + // See [[CDCReader]] for general details on how partitioning on the CDC type column works. + + // In the following two functions `matchedClauseOutput` and `notMatchedClauseOutput`, we + // produce a Seq[Expression] for each intended output row. + // Depending on the clause and whether CDC is enabled, we output between 0 and 3 rows, as a + // Seq[Seq[Expression]] + + // There is one corner case outlined above at isDeleteWithDuplicateMatchesAndCdc definition. + // When we have a delete-ONLY merge with duplicate matches we have N + 4 columns: + // N target cols, TARGET_ROW_ID_COL, ROW_DROPPED_COL, INCR_ROW_COUNT_COL, CDC_TYPE_COLUMN_NAME + // When we have a delete-when-matched merge with duplicate matches + an insert clause, we have + // N + 5 columns: + // N target cols, TARGET_ROW_ID_COL, SOURCE_ROW_ID_COL, ROW_DROPPED_COL, INCR_ROW_COUNT_COL, + // CDC_TYPE_COLUMN_NAME + // These ROW_ID_COL will always be dropped before the final write. + + if (isDeleteWithDuplicateMatchesAndCdc) { + targetOutputCols = targetOutputCols :+ UnresolvedAttribute(TARGET_ROW_ID_COL) + outputRowSchema = outputRowSchema.add(TARGET_ROW_ID_COL, DataTypes.LongType) + if (notMatchedClauses.nonEmpty) { // there is an insert clause, make SRC_ROW_ID_COL=null + targetOutputCols = targetOutputCols :+ Alias(Literal(null), SOURCE_ROW_ID_COL)() + outputRowSchema = outputRowSchema.add(SOURCE_ROW_ID_COL, DataTypes.LongType) + } + } + + if (cdcEnabled) { + outputRowSchema = outputRowSchema + .add(ROW_DROPPED_COL, DataTypes.BooleanType) + .add(INCR_ROW_COUNT_COL, DataTypes.BooleanType) + .add(CDC_TYPE_COLUMN_NAME, DataTypes.StringType) + } + + def matchedClauseOutput(clause: DeltaMergeIntoMatchedClause): Seq[Seq[Expression]] = { + val exprs = clause match { + case u: DeltaMergeIntoMatchedUpdateClause => + // Generate update expressions and set ROW_DELETED_COL = false and + // CDC_TYPE_COLUMN_NAME = CDC_TYPE_NOT_CDC + val mainDataOutput = u.resolvedActions.map(_.expr) :+ FalseLiteral :+ + incrUpdatedCountExpr :+ CDC_TYPE_NOT_CDC_LITERAL + if (cdcEnabled) { + // For update preimage, we have do a no-op copy with ROW_DELETED_COL = false and + // CDC_TYPE_COLUMN_NAME = CDC_TYPE_UPDATE_PREIMAGE and INCR_ROW_COUNT_COL as a no-op + // (because the metric will be incremented in `mainDataOutput`) + val preImageOutput = targetOutputCols :+ FalseLiteral :+ TrueLiteral :+ + Literal(CDC_TYPE_UPDATE_PREIMAGE) + // For update postimage, we have the same expressions as for mainDataOutput but with + // INCR_ROW_COUNT_COL as a no-op (because the metric will be incremented in + // `mainDataOutput`), and CDC_TYPE_COLUMN_NAME = CDC_TYPE_UPDATE_POSTIMAGE + val postImageOutput = mainDataOutput.dropRight(2) :+ TrueLiteral :+ + Literal(CDC_TYPE_UPDATE_POSTIMAGE) + Seq(mainDataOutput, preImageOutput, postImageOutput) + } else { + Seq(mainDataOutput) + } + case _: DeltaMergeIntoMatchedDeleteClause => + // Generate expressions to set the ROW_DELETED_COL = true and CDC_TYPE_COLUMN_NAME = + // CDC_TYPE_NOT_CDC + val mainDataOutput = targetOutputCols :+ TrueLiteral :+ incrDeletedCountExpr :+ + CDC_TYPE_NOT_CDC_LITERAL + if (cdcEnabled) { + // For delete we do a no-op copy with ROW_DELETED_COL = false, INCR_ROW_COUNT_COL as a + // no-op (because the metric will be incremented in `mainDataOutput`) and + // CDC_TYPE_COLUMN_NAME = CDC_TYPE_DELETE + val deleteCdcOutput = targetOutputCols :+ FalseLiteral :+ TrueLiteral :+ + Literal(CDC_TYPE_DELETE) + Seq(mainDataOutput, deleteCdcOutput) + } else { + Seq(mainDataOutput) + } + } + exprs.map(resolveOnJoinedPlan) + } + + def notMatchedClauseOutput(clause: DeltaMergeIntoNotMatchedClause): Seq[Seq[Expression]] = { + // Generate insert expressions and set ROW_DELETED_COL = false and + // CDC_TYPE_COLUMN_NAME = CDC_TYPE_NOT_CDC + val insertExprs = clause.resolvedActions.map(_.expr) + val mainDataOutput = resolveOnJoinedPlan( + if (isDeleteWithDuplicateMatchesAndCdc) { + // Must be delete-when-matched merge with duplicate matches + insert clause + // Therefore we must keep the target row id and source row id. Since this is a not-matched + // clause we know the target row-id will be null. See above at + // isDeleteWithDuplicateMatchesAndCdc definition for more details. + insertExprs :+ + Alias(Literal(null), TARGET_ROW_ID_COL)() :+ UnresolvedAttribute(SOURCE_ROW_ID_COL) :+ + FalseLiteral :+ incrInsertedCountExpr :+ CDC_TYPE_NOT_CDC_LITERAL + } else { + insertExprs :+ FalseLiteral :+ incrInsertedCountExpr :+ CDC_TYPE_NOT_CDC_LITERAL + } + ) + if (cdcEnabled) { + // For insert we have the same expressions as for mainDataOutput, but with + // INCR_ROW_COUNT_COL as a no-op (because the metric will be incremented in + // `mainDataOutput`), and CDC_TYPE_COLUMN_NAME = CDC_TYPE_INSERT + val insertCdcOutput = mainDataOutput.dropRight(2) :+ TrueLiteral :+ Literal(CDC_TYPE_INSERT) + Seq(mainDataOutput, insertCdcOutput) + } else { + Seq(mainDataOutput) + } + } + + def clauseCondition(clause: DeltaMergeIntoClause): Expression = { + // if condition is None, then expression always evaluates to true + val condExpr = clause.condition.getOrElse(TrueLiteral) + resolveOnJoinedPlan(Seq(condExpr)).head + } + + val targetRowHasNoMatch = resolveOnJoinedPlan(Seq(col(SOURCE_ROW_PRESENT_COL).isNull.expr)).head + val sourceRowHasNoMatch = resolveOnJoinedPlan(Seq(col(TARGET_ROW_PRESENT_COL).isNull.expr)).head + val matchedConditions = matchedClauses.map(clauseCondition) + val matchedOutputs = matchedClauses.map(matchedClauseOutput) + val notMatchedConditions = notMatchedClauses.map(clauseCondition) + val notMatchedOutputs = notMatchedClauses.map(notMatchedClauseOutput) + // TODO support notMatchedBySourceClauses which is new in DBR 12.2 + // https://github.com/NVIDIA/spark-rapids/issues/8415 + val notMatchedBySourceConditions = Seq.empty + val notMatchedBySourceOutputs = Seq.empty + val noopCopyOutput = + resolveOnJoinedPlan(targetOutputCols :+ FalseLiteral :+ incrNoopCountExpr :+ + CDC_TYPE_NOT_CDC_LITERAL) + val deleteRowOutput = + resolveOnJoinedPlan(targetOutputCols :+ TrueLiteral :+ TrueLiteral :+ + CDC_TYPE_NOT_CDC_LITERAL) + var outputDF = addMergeJoinProcessor(spark, joinedPlan, outputRowSchema, + targetRowHasNoMatch = targetRowHasNoMatch, + sourceRowHasNoMatch = sourceRowHasNoMatch, + matchedConditions = matchedConditions, + matchedOutputs = matchedOutputs, + notMatchedConditions = notMatchedConditions, + notMatchedOutputs = notMatchedOutputs, + notMatchedBySourceConditions = notMatchedBySourceConditions, + notMatchedBySourceOutputs = notMatchedBySourceOutputs, + noopCopyOutput = noopCopyOutput, + deleteRowOutput = deleteRowOutput) + + if (isDeleteWithDuplicateMatchesAndCdc) { + // When we have a delete when matched clause with duplicate matches we have to remove + // duplicate CDC rows. This scenario is further explained at + // isDeleteWithDuplicateMatchesAndCdc definition. + + // To remove duplicate CDC rows generated by the duplicate matches we dedupe by + // TARGET_ROW_ID_COL since there should only be one CDC delete row per target row. + // When there is an insert clause in addition to the delete clause we additionally dedupe by + // SOURCE_ROW_ID_COL and CDC_TYPE_COLUMN_NAME to avoid dropping valid duplicate inserted rows + // and their corresponding CDC rows. + val columnsToDedupeBy = if (notMatchedClauses.nonEmpty) { // insert clause + Seq(TARGET_ROW_ID_COL, SOURCE_ROW_ID_COL, CDC_TYPE_COLUMN_NAME) + } else { + Seq(TARGET_ROW_ID_COL) + } + outputDF = outputDF + .dropDuplicates(columnsToDedupeBy) + .drop(ROW_DROPPED_COL, INCR_ROW_COUNT_COL, TARGET_ROW_ID_COL, SOURCE_ROW_ID_COL) + } else { + outputDF = outputDF.drop(ROW_DROPPED_COL, INCR_ROW_COUNT_COL) + } + + logDebug("writeAllChanges: join output plan:\n" + outputDF.queryExecution) + + // Write to Delta + val newFiles = deltaTxn + .writeFiles(repartitionIfNeeded(spark, outputDF, deltaTxn.metadata.partitionColumns)) + + // Update metrics + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + metrics("numTargetChangeFilesAdded") += newFiles.count(_.isInstanceOf[AddCDCFile]) + metrics("numTargetChangeFileBytes") += newFiles.collect{ case f: AddCDCFile => f.size }.sum + metrics("numTargetBytesAdded") += addedBytes + metrics("numTargetPartitionsAddedTo") += addedPartitions + if (multipleMatchDeleteOnlyOvercount.isDefined) { + // Compensate for counting duplicates during the query. + val actualRowsDeleted = + metrics("numTargetRowsDeleted").value - multipleMatchDeleteOnlyOvercount.get + assert(actualRowsDeleted >= 0) + metrics("numTargetRowsDeleted").set(actualRowsDeleted) + } + + newFiles + } + + private def addMergeJoinProcessor( + spark: SparkSession, + joinedPlan: LogicalPlan, + outputRowSchema: StructType, + targetRowHasNoMatch: Expression, + sourceRowHasNoMatch: Expression, + matchedConditions: Seq[Expression], + matchedOutputs: Seq[Seq[Seq[Expression]]], + notMatchedConditions: Seq[Expression], + notMatchedOutputs: Seq[Seq[Seq[Expression]]], + notMatchedBySourceConditions: Seq[Expression], + notMatchedBySourceOutputs: Seq[Seq[Seq[Expression]]], + noopCopyOutput: Seq[Expression], + deleteRowOutput: Seq[Expression]): Dataset[Row] = { + def wrap(e: Expression): BaseExprMeta[Expression] = { + GpuOverrides.wrapExpr(e, rapidsConf, None) + } + + val targetRowHasNoMatchMeta = wrap(targetRowHasNoMatch) + val sourceRowHasNoMatchMeta = wrap(sourceRowHasNoMatch) + val matchedConditionsMetas = matchedConditions.map(wrap) + val matchedOutputsMetas = matchedOutputs.map(_.map(_.map(wrap))) + val notMatchedConditionsMetas = notMatchedConditions.map(wrap) + val notMatchedOutputsMetas = notMatchedOutputs.map(_.map(_.map(wrap))) + val notMatchedBySourceConditionsMetas = notMatchedBySourceConditions.map(wrap) + val notMatchedBySourceOutputsMetas = notMatchedBySourceOutputs.map(_.map(_.map(wrap))) + val noopCopyOutputMetas = noopCopyOutput.map(wrap) + val deleteRowOutputMetas = deleteRowOutput.map(wrap) + val allMetas = Seq(targetRowHasNoMatchMeta, sourceRowHasNoMatchMeta) ++ + matchedConditionsMetas ++ matchedOutputsMetas.flatten.flatten ++ + notMatchedConditionsMetas ++ notMatchedOutputsMetas.flatten.flatten ++ + notMatchedBySourceConditionsMetas ++ notMatchedBySourceOutputsMetas.flatten.flatten ++ + noopCopyOutputMetas ++ deleteRowOutputMetas + allMetas.foreach(_.tagForGpu()) + val canReplace = allMetas.forall(_.canExprTreeBeReplaced) && rapidsConf.isOperatorEnabled( + "spark.rapids.sql.exec.RapidsProcessDeltaMergeJoinExec", false, false) + if (rapidsConf.shouldExplainAll || (rapidsConf.shouldExplain && !canReplace)) { + val exprExplains = allMetas.map(_.explain(rapidsConf.shouldExplainAll)) + val execWorkInfo = if (canReplace) { + "will run on GPU" + } else { + "cannot run on GPU because not all merge processing expressions can be replaced" + } + logWarning(s" $execWorkInfo:\n" + + s" ${exprExplains.mkString(" ")}") + } + + if (canReplace) { + val processedJoinPlan = RapidsProcessDeltaMergeJoin( + joinedPlan, + toAttributes(outputRowSchema), + targetRowHasNoMatch = targetRowHasNoMatch, + sourceRowHasNoMatch = sourceRowHasNoMatch, + matchedConditions = matchedConditions, + matchedOutputs = matchedOutputs, + notMatchedConditions = notMatchedConditions, + notMatchedOutputs = notMatchedOutputs, + notMatchedBySourceConditions = notMatchedBySourceConditions, + notMatchedBySourceOutputs = notMatchedBySourceOutputs, + noopCopyOutput = noopCopyOutput, + deleteRowOutput = deleteRowOutput) + Dataset.ofRows(spark, processedJoinPlan) + } else { + val joinedRowEncoder = ExpressionEncoder(RowEncoder.encoderFor(joinedPlan.schema)) + val outputRowEncoder = ExpressionEncoder(RowEncoder.encoderFor(outputRowSchema)). + resolveAndBind() + + val processor = new JoinedRowProcessor( + targetRowHasNoMatch = targetRowHasNoMatch, + sourceRowHasNoMatch = sourceRowHasNoMatch, + matchedConditions = matchedConditions, + matchedOutputs = matchedOutputs, + notMatchedConditions = notMatchedConditions, + notMatchedOutputs = notMatchedOutputs, + noopCopyOutput = noopCopyOutput, + deleteRowOutput = deleteRowOutput, + joinedAttributes = joinedPlan.output, + joinedRowEncoder = joinedRowEncoder, + outputRowEncoder = outputRowEncoder) + + Dataset.ofRows(spark, joinedPlan).mapPartitions(processor.processPartition)(outputRowEncoder) + } + } + + /** + * Build a new logical plan using the given `files` that has the same output columns (exprIds) + * as the `target` logical plan, so that existing update/insert expressions can be applied + * on this new plan. + */ + private def buildTargetPlanWithFiles( + deltaTxn: OptimisticTransaction, + files: Seq[AddFile]): LogicalPlan = { + val targetOutputCols = getTargetOutputCols(deltaTxn) + val targetOutputColsMap = { + val colsMap: Map[String, NamedExpression] = targetOutputCols.view + .map(col => col.name -> col).toMap + if (conf.caseSensitiveAnalysis) { + colsMap + } else { + CaseInsensitiveMap(colsMap) + } + } + + val plan = { + // We have to do surgery to use the attributes from `targetOutputCols` to scan the table. + // In cases of schema evolution, they may not be the same type as the original attributes. + val original = + deltaTxn.deltaLog.createDataFrame(deltaTxn.snapshot, files).queryExecution.analyzed + val transformed = original.transform { + case LogicalRelation(base, _, catalogTbl, isStreaming) => + LogicalRelation( + base, + // We can ignore the new columns which aren't yet AttributeReferences. + targetOutputCols.collect { case a: AttributeReference => a }, + catalogTbl, + isStreaming) + } + + // In case of schema evolution & column mapping, we would also need to rebuild the file format + // because under column mapping, the reference schema within DeltaParquetFileFormat + // that is used to populate metadata needs to be updated + if (deltaTxn.metadata.columnMappingMode != NoMapping) { + val updatedFileFormat = deltaTxn.deltaLog.fileFormat( + deltaTxn.deltaLog.unsafeVolatileSnapshot.protocol, deltaTxn.metadata) + DeltaTableUtils.replaceFileFormat(transformed, updatedFileFormat) + } else { + transformed + } + } + + // For each plan output column, find the corresponding target output column (by name) and + // create an alias + val aliases = plan.output.map { + case newAttrib: AttributeReference => + val existingTargetAttrib = targetOutputColsMap.get(newAttrib.name) + .getOrElse { + throw new AnalysisException( + s"Could not find ${newAttrib.name} among the existing target output " + + targetOutputCols.mkString(",")) + }.asInstanceOf[AttributeReference] + + if (existingTargetAttrib.exprId == newAttrib.exprId) { + // It's not valid to alias an expression to its own exprId (this is considered a + // non-unique exprId by the analyzer), so we just use the attribute directly. + newAttrib + } else { + Alias(newAttrib, existingTargetAttrib.name)(exprId = existingTargetAttrib.exprId) + } + } + + Project(aliases, plan) + } + + /** Expressions to increment SQL metrics */ + private def makeMetricUpdateUDF(name: String, deterministic: Boolean = false): Expression = { + // only capture the needed metric in a local variable + val metric = metrics(name) + var u = udf(new GpuDeltaMetricUpdateUDF(metric)) + if (!deterministic) { + u = u.asNondeterministic() + } + u.apply().expr + } + + private def getTargetOutputCols(txn: OptimisticTransaction): Seq[NamedExpression] = { + txn.metadata.schema.map { col => + targetOutputAttributesMap + .get(col.name) + .map { a => + AttributeReference(col.name, col.dataType, col.nullable)(a.exprId) + } + .getOrElse(Alias(Literal(null), col.name)() + ) + } + } + + /** + * Repartitions the output DataFrame by the partition columns if table is partitioned + * and `merge.repartitionBeforeWrite.enabled` is set to true. + */ + protected def repartitionIfNeeded( + spark: SparkSession, + df: DataFrame, + partitionColumns: Seq[String]): DataFrame = { + if (partitionColumns.nonEmpty && spark.conf.get(DeltaSQLConf.MERGE_REPARTITION_BEFORE_WRITE)) { + df.repartition(partitionColumns.map(col): _*) + } else { + df + } + } + + /** + * Execute the given `thunk` and return its result while recording the time taken to do it. + * + * @param sqlMetricName name of SQL metric to update with the time taken by the thunk + * @param thunk the code to execute + */ + private def recordMergeOperation[A](sqlMetricName: String)(thunk: => A): A = { + val startTimeNs = System.nanoTime() + val r = thunk + val timeTakenMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + if (sqlMetricName != null && timeTakenMs > 0) { + metrics(sqlMetricName) += timeTakenMs + } + r + } +} + +object GpuMergeIntoCommand { + /** + * Spark UI will track all normal accumulators along with Spark tasks to show them on Web UI. + * However, the accumulator used by `MergeIntoCommand` can store a very large value since it + * tracks all files that need to be rewritten. We should ask Spark UI to not remember it, + * otherwise, the UI data may consume lots of memory. Hence, we use the prefix `internal.metrics.` + * to make this accumulator become an internal accumulator, so that it will not be tracked by + * Spark UI. + */ + val TOUCHED_FILES_ACCUM_NAME = "internal.metrics.MergeIntoDelta.touchedFiles" + + val ROW_ID_COL = "_row_id_" + val TARGET_ROW_ID_COL = "_target_row_id_" + val SOURCE_ROW_ID_COL = "_source_row_id_" + val FILE_NAME_COL = "_file_name_" + val SOURCE_ROW_PRESENT_COL = "_source_row_present_" + val TARGET_ROW_PRESENT_COL = "_target_row_present_" + val ROW_DROPPED_COL = GpuDeltaMergeConstants.ROW_DROPPED_COL + val INCR_ROW_COUNT_COL = "_incr_row_count_" + + // Some Delta versions use Literal(null) which translates to a literal of NullType instead + // of the Literal(null, StringType) which is needed, so using a fixed version here + // rather than the version from Delta Lake. + val CDC_TYPE_NOT_CDC_LITERAL = Literal(null, StringType) + + /** + * @param targetRowHasNoMatch whether a joined row is a target row with no match in the source + * table + * @param sourceRowHasNoMatch whether a joined row is a source row with no match in the target + * table + * @param matchedConditions condition for each match clause + * @param matchedOutputs corresponding output for each match clause. for each clause, we + * have 1-3 output rows, each of which is a sequence of expressions + * to apply to the joined row + * @param notMatchedConditions condition for each not-matched clause + * @param notMatchedOutputs corresponding output for each not-matched clause. for each clause, + * we have 1-2 output rows, each of which is a sequence of + * expressions to apply to the joined row + * @param noopCopyOutput no-op expression to copy a target row to the output + * @param deleteRowOutput expression to drop a row from the final output. this is used for + * source rows that don't match any not-matched clauses + * @param joinedAttributes schema of our outer-joined dataframe + * @param joinedRowEncoder joinedDF row encoder + * @param outputRowEncoder final output row encoder + */ + class JoinedRowProcessor( + targetRowHasNoMatch: Expression, + sourceRowHasNoMatch: Expression, + matchedConditions: Seq[Expression], + matchedOutputs: Seq[Seq[Seq[Expression]]], + notMatchedConditions: Seq[Expression], + notMatchedOutputs: Seq[Seq[Seq[Expression]]], + noopCopyOutput: Seq[Expression], + deleteRowOutput: Seq[Expression], + joinedAttributes: Seq[Attribute], + joinedRowEncoder: ExpressionEncoder[Row], + outputRowEncoder: ExpressionEncoder[Row]) extends Serializable { + + private def generateProjection(exprs: Seq[Expression]): UnsafeProjection = { + UnsafeProjection.create(exprs, joinedAttributes) + } + + private def generatePredicate(expr: Expression): BasePredicate = { + GeneratePredicate.generate(expr, joinedAttributes) + } + + def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { + + val targetRowHasNoMatchPred = generatePredicate(targetRowHasNoMatch) + val sourceRowHasNoMatchPred = generatePredicate(sourceRowHasNoMatch) + val matchedPreds = matchedConditions.map(generatePredicate) + val matchedProjs = matchedOutputs.map(_.map(generateProjection)) + val notMatchedPreds = notMatchedConditions.map(generatePredicate) + val notMatchedProjs = notMatchedOutputs.map(_.map(generateProjection)) + val noopCopyProj = generateProjection(noopCopyOutput) + val deleteRowProj = generateProjection(deleteRowOutput) + val outputProj = UnsafeProjection.create(outputRowEncoder.schema) + + // this is accessing ROW_DROPPED_COL. If ROW_DROPPED_COL is not in outputRowEncoder.schema + // then CDC must be disabled and it's the column after our output cols + def shouldDeleteRow(row: InternalRow): Boolean = { + row.getBoolean( + outputRowEncoder.schema.getFieldIndex(ROW_DROPPED_COL) + .getOrElse(outputRowEncoder.schema.fields.size) + ) + } + + def processRow(inputRow: InternalRow): Iterator[InternalRow] = { + if (targetRowHasNoMatchPred.eval(inputRow)) { + // Target row did not match any source row, so just copy it to the output + Iterator(noopCopyProj.apply(inputRow)) + } else { + // identify which set of clauses to execute: matched or not-matched ones + val (predicates, projections, noopAction) = if (sourceRowHasNoMatchPred.eval(inputRow)) { + // Source row did not match with any target row, so insert the new source row + (notMatchedPreds, notMatchedProjs, deleteRowProj) + } else { + // Source row matched with target row, so update the target row + (matchedPreds, matchedProjs, noopCopyProj) + } + + // find (predicate, projection) pair whose predicate satisfies inputRow + val pair = (predicates zip projections).find { + case (predicate, _) => predicate.eval(inputRow) + } + + pair match { + case Some((_, projections)) => + projections.map(_.apply(inputRow)).iterator + case None => Iterator(noopAction.apply(inputRow)) + } + } + } + + val toRow = joinedRowEncoder.createSerializer() + val fromRow = outputRowEncoder.createDeserializer() + rowIterator + .map(toRow) + .flatMap(processRow) + .filter(!shouldDeleteRow(_)) + .map { notDeletedInternalRow => + fromRow(outputProj(notDeletedInternalRow)) + } + } + } + + /** Count the number of distinct partition values among the AddFiles in the given set. */ + def totalBytesAndDistinctPartitionValues(files: Seq[FileAction]): (Long, Int) = { + val distinctValues = new mutable.HashSet[Map[String, String]]() + var bytes = 0L + val iter = files.collect { case a: AddFile => a }.iterator + while (iter.hasNext) { + val file = iter.next() + distinctValues += file.partitionValues + bytes += file.size + } + // If the only distinct value map is an empty map, then it must be an unpartitioned table. + // Return 0 in that case. + val numDistinctValues = + if (distinctValues.size == 1 && distinctValues.head.isEmpty) 0 else distinctValues.size + (bytes, numDistinctValues) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala new file mode 100644 index 00000000000..3e836056b6d --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -0,0 +1,312 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import java.net.URI + +import scala.collection.mutable.ListBuffer + +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.actions.{AddFile, FileAction} +import com.databricks.sql.transaction.tahoe.constraints.{Constraint, Constraints} +import com.databricks.sql.transaction.tahoe.schema.InvariantViolationException +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.delta._ +import org.apache.commons.lang3.exception.ExceptionUtils +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormatWriter} +import org.apache.spark.sql.functions.to_json +import org.apache.spark.sql.rapids.{BasicColumnarWriteJobStatsTracker, ColumnarWriteJobStatsTracker, GpuFileFormatWriter, GpuWriteJobStatsTracker} +import org.apache.spark.sql.rapids.delta.GpuIdentityColumn +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.{Clock, SerializableConfiguration} + +/** + * Used to perform a set of reads in a transaction and then commit a set of updates to the + * state of the log. All reads from the DeltaLog, MUST go through this instance rather + * than directly to the DeltaLog otherwise they will not be check for logical conflicts + * with concurrent updates. + * + * This class is not thread-safe. + * + * @param deltaLog The Delta Log for the table this transaction is modifying. + * @param snapshot The snapshot that this transaction is reading at. + * @param rapidsConf RAPIDS Accelerator config settings. + */ +class GpuOptimisticTransaction( + deltaLog: DeltaLog, + snapshot: Snapshot, + rapidsConf: RapidsConf)(implicit clock: Clock) + extends GpuOptimisticTransactionBase(deltaLog, snapshot, rapidsConf)(clock) { + + /** Creates a new OptimisticTransaction. + * + * @param deltaLog The Delta Log for the table this transaction is modifying. + * @param rapidsConf RAPIDS Accelerator config settings + */ + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { + this(deltaLog, deltaLog.update(), rapidsConf) + } + + private def getGpuStatsColExpr( + statsDataSchema: Seq[Attribute], + statsCollection: GpuStatisticsCollection): Expression = { + Dataset.ofRows(spark, LocalRelation(statsDataSchema)) + .select(to_json(statsCollection.statsCollector)) + .queryExecution.analyzed.expressions.head + } + + /** Return the pair of optional stats tracker and stats collection class */ + private def getOptionalGpuStatsTrackerAndStatsCollection( + output: Seq[Attribute], + partitionSchema: StructType, data: DataFrame): ( + Option[GpuDeltaJobStatisticsTracker], + Option[GpuStatisticsCollection]) = { + if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_COLLECT_STATS)) { + + val (statsDataSchema, statsCollectionSchema) = getStatsSchema(output, partitionSchema) + + val indexedCols = DeltaConfigs.DATA_SKIPPING_NUM_INDEXED_COLS.fromMetaData(metadata) + val prefixLength = + spark.sessionState.conf.getConf(DeltaSQLConf.DATA_SKIPPING_STRING_PREFIX_LENGTH) + val tableSchema = { + // If collecting stats using the table schema, then pass in statsCollectionSchema. + // Otherwise pass in statsDataSchema to collect stats using the DataFrame schema. + if (spark.sessionState.conf.getConf(DeltaSQLConf + .DELTA_COLLECT_STATS_USING_TABLE_SCHEMA)) { + statsCollectionSchema.toStructType + } else { + statsDataSchema.toStructType + } + } + + val _spark = spark + val protocol = deltaLog.unsafeVolatileSnapshot.protocol + + val statsCollection = new GpuStatisticsCollection { + override val spark = _spark + override val deletionVectorsSupported = + protocol.isFeatureSupported(DeletionVectorsTableFeature) + override val tableDataSchema = tableSchema + override val dataSchema = statsDataSchema.toStructType + override val numIndexedCols = indexedCols + override val stringPrefixLength: Int = prefixLength + } + + val statsColExpr = getGpuStatsColExpr(statsDataSchema, statsCollection) + + val statsSchema = statsCollection.statCollectionSchema + val explodedDataSchema = statsCollection.explodedDataSchema + val batchStatsToRow = (batch: ColumnarBatch, row: InternalRow) => { + GpuStatisticsCollection.batchStatsToRow(statsSchema, explodedDataSchema, batch, row) + } + (Some(new GpuDeltaJobStatisticsTracker(statsDataSchema, statsColExpr, batchStatsToRow)), + Some(statsCollection)) + } else { + (None, None) + } + } + + override def writeFiles( + inputData: Dataset[_], + writeOptions: Option[DeltaOptions], + additionalConstraints: Seq[Constraint]): Seq[FileAction] = { + hasWritten = true + + val spark = inputData.sparkSession + val (data, partitionSchema) = performCDCPartition(inputData) + val outputPath = deltaLog.dataPath + + val (normalizedQueryExecution, output, generatedColumnConstraints, dataHighWaterMarks) = { + // TODO: is none ok to pass here? + normalizeData(deltaLog, None, data) + } + val highWaterMarks = trackHighWaterMarks.getOrElse(dataHighWaterMarks) + + // Build a new plan with a stub GpuDeltaWrite node to work around undesired transitions between + // columns and rows when AQE is involved. Without this node in the plan, AdaptiveSparkPlanExec + // could be the root node of the plan. In that case we do not have enough context to know + // whether the AdaptiveSparkPlanExec should be columnar or not, since the GPU overrides do not + // see how the parent is using the AdaptiveSparkPlanExec outputs. By using this stub node that + // appears to be a data writing node to AQE (it derives from V2CommandExec), the + // AdaptiveSparkPlanExec will be planned as a child of this new node. That provides enough + // context to plan the AQE sub-plan properly with respect to columnar and row transitions. + // We could force the AQE node to be columnar here by explicitly replacing the node, but that + // breaks the connection between the queryExecution and the node that will actually execute. + val gpuWritePlan = Dataset.ofRows(spark, RapidsDeltaWrite(normalizedQueryExecution.logical)) + val queryExecution = gpuWritePlan.queryExecution + + val partitioningColumns = getPartitioningColumns(partitionSchema, output) + + val committer = getCommitter(outputPath) + + // If Statistics Collection is enabled, then create a stats tracker that will be injected during + // the FileFormatWriter.write call below and will collect per-file stats using + // StatisticsCollection + val (optionalStatsTracker, _) = getOptionalGpuStatsTrackerAndStatsCollection(output, + partitionSchema, data) + + // schema should be normalized, therefore we can do an equality check + val (statsDataSchema, _) = getStatsSchema(output, partitionSchema) + val identityTracker = GpuIdentityColumn.createIdentityColumnStatsTracker( + spark, + statsDataSchema, + metadata.schema, + highWaterMarks) + + val constraints = + Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints + + val isOptimize = isOptimizeCommand(queryExecution.analyzed) + + SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) { + val outputSpec = FileFormatWriter.OutputSpec( + outputPath.toString, + Map.empty, + output) + + // Remove any unnecessary row conversions added as part of Spark planning + val queryPhysicalPlan = queryExecution.executedPlan match { + case GpuColumnarToRowExec(child, _) => child + case p => p + } + val gpuRapidsWrite = queryPhysicalPlan match { + case g: GpuRapidsDeltaWriteExec => Some(g) + case _ => None + } + + val empty2NullPlan = convertEmptyToNullIfNeeded(queryPhysicalPlan, + partitioningColumns, constraints) + val optimizedPlan = + applyOptimizeWriteIfNeeded(spark, empty2NullPlan, partitionSchema, isOptimize) + val planWithInvariants = addInvariantChecks(optimizedPlan, constraints) + val physicalPlan = convertToGpu(planWithInvariants) + + val statsTrackers: ListBuffer[ColumnarWriteJobStatsTracker] = ListBuffer() + + val hadoopConf = spark.sessionState.newHadoopConfWithOptions( + metadata.configuration ++ deltaLog.options) + + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + val serializableHadoopConf = new SerializableConfiguration(hadoopConf) + val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( + serializableHadoopConf, + BasicWriteJobStatsTracker.metrics) + registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + statsTrackers.append(basicWriteJobStatsTracker) + gpuRapidsWrite.foreach { grw => + val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, + grw.basicMetrics, grw.taskMetrics) + statsTrackers.append(tracker) + } + } + + // Retain only a minimal selection of Spark writer options to avoid any potential + // compatibility issues + val options = writeOptions match { + case None => Map.empty[String, String] + case Some(writeOptions) => + writeOptions.options.filterKeys { key => + key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) || + key.equalsIgnoreCase(DeltaOptions.COMPRESSION) + }.toMap + } + val deltaFileFormat = deltaLog.fileFormat(deltaLog.unsafeVolatileSnapshot.protocol, metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") + } + + try { + logDebug(s"Physical plan for write:\n$physicalPlan") + GpuFileFormatWriter.write( + sparkSession = spark, + plan = physicalPlan, + fileFormat = gpuFileFormat, + committer = committer, + outputSpec = outputSpec, + hadoopConf = hadoopConf, + partitionColumns = partitioningColumns, + bucketSpec = None, + statsTrackers = optionalStatsTracker.toSeq ++ identityTracker.toSeq ++ statsTrackers, + options = options, + rapidsConf.stableSort, + rapidsConf.concurrentWriterPartitionFlushSize) + } catch { + case s: SparkException => + // Pull an InvariantViolationException up to the top level if it was the root cause. + val violationException = ExceptionUtils.getRootCause(s) + if (violationException.isInstanceOf[InvariantViolationException]) { + throw violationException + } else { + throw s + } + } + } + + val resultFiles = committer.addedStatuses.map { a => + a.copy(stats = optionalStatsTracker.map( + _.recordedStats(new Path(new URI(a.path)).getName)).getOrElse(a.stats)) + }.filter { + // In some cases, we can write out an empty `inputData`. Some examples of this (though, they + // may be fixed in the future) are the MERGE command when you delete with empty source, or + // empty target, or on disjoint tables. This is hard to catch before the write without + // collecting the DF ahead of time. Instead, we can return only the AddFiles that + // a) actually add rows, or + // b) don't have any stats so we don't know the number of rows at all + case a: AddFile => a.numLogicalRecords.forall(_ > 0) + case _ => true + } + + identityTracker.foreach { tracker => + updatedIdentityHighWaterMarks.appendAll(tracker.highWaterMarks.toSeq) + } + val fileActions = resultFiles.toSeq ++ committer.changeFiles + + // Check if auto-compaction is enabled. + // (Auto compaction checks are derived from the work in + // https://github.com/delta-io/delta/pull/1156). + lazy val autoCompactEnabled = + spark.sessionState.conf + .getConf[String](DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED) + .getOrElse { + DeltaConfigs.AUTO_COMPACT.fromMetaData(metadata) + .getOrElse("false") + }.toBoolean + + if (!isOptimize && autoCompactEnabled && fileActions.nonEmpty) { + registerPostCommitHook(GpuDoAutoCompaction) + } + + fileActions + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala new file mode 100644 index 00000000000..26efbcd46de --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransactionBase.scala @@ -0,0 +1,194 @@ +/* + * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * + * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.actions.FileAction +import com.databricks.sql.transaction.tahoe.constraints.{Constraint, DeltaInvariantCheckerExec} +import com.databricks.sql.transaction.tahoe.files.TahoeBatchFileIndex +import com.databricks.sql.transaction.tahoe.metering.DeltaLogging +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.rapids.GpuShuffleEnv +import org.apache.spark.sql.rapids.GpuV1WriteUtils.GpuEmpty2Null +import org.apache.spark.sql.rapids.delta.{DeltaShufflePartitionsUtil, GpuOptimizeWriteExchangeExec, OptimizeWriteExchangeExec} +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.Clock + +/** + * Used to perform a set of reads in a transaction and then commit a set of updates to the + * state of the log. All reads from the DeltaLog, MUST go through this instance rather + * than directly to the DeltaLog otherwise they will not be check for logical conflicts + * with concurrent updates. + * + * This class is not thread-safe. + * + * @param deltaLog The Delta Log for the table this transaction is modifying. + * @param snapshot The snapshot that this transaction is reading at. + * @param rapidsConf RAPIDS Accelerator config settings. + */ +abstract class GpuOptimisticTransactionBase + (deltaLog: DeltaLog, snapshot: Snapshot, val rapidsConf: RapidsConf) + (implicit clock: Clock) + extends OptimisticTransaction(deltaLog, Option.empty[CatalogTable], snapshot) + with DeltaLogging { + + /** + * Adds checking of constraints on the table + * @param plan Plan to generate the table to check against constraints + * @param constraints Constraints to check on the table + * @return GPU columnar plan to execute + */ + protected def addInvariantChecks(plan: SparkPlan, constraints: Seq[Constraint]): SparkPlan = { + val cpuInvariants = + DeltaInvariantCheckerExec.buildInvariantChecks(plan.output, constraints, plan.session) + GpuCheckDeltaInvariant.maybeConvertToGpu(cpuInvariants, rapidsConf) match { + case Some(gpuInvariants) => + val gpuPlan = convertToGpu(plan) + GpuDeltaInvariantCheckerExec(gpuPlan, gpuInvariants) + case None => + val cpuPlan = convertToCpu(plan) + DeltaInvariantCheckerExec(cpuPlan, constraints) + } + } + + /** GPU version of convertEmptyToNullIfNeeded */ + private def gpuConvertEmptyToNullIfNeeded( + plan: GpuExec, + partCols: Seq[Attribute], + constraints: Seq[Constraint]): SparkPlan = { + if (!spark.conf.get(DeltaSQLConf.CONVERT_EMPTY_TO_NULL_FOR_STRING_PARTITION_COL)) { + return plan + } + // No need to convert if there are no constraints. The empty strings will be converted later by + // FileFormatWriter and FileFormatDataWriter. Note that we might still do unnecessary convert + // here as the constraints might not be related to the string partition columns. A precise + // check will need to walk the constraints to see if such columns are really involved. It + // doesn't seem to worth the effort. + if (constraints.isEmpty) return plan + + val partSet = AttributeSet(partCols) + var needConvert = false + val projectList: Seq[NamedExpression] = plan.output.map { + case p if partSet.contains(p) && p.dataType == StringType => + needConvert = true + GpuAlias(GpuEmpty2Null(p), p.name)() + case attr => attr + } + if (needConvert) GpuProjectExec(projectList.toList, plan) else plan + } + + /** + * If there is any string partition column and there are constraints defined, add a projection to + * convert empty string to null for that column. The empty strings will be converted to null + * eventually even without this convert, but we want to do this earlier before check constraints + * so that empty strings are correctly rejected. Note that this should not cause the downstream + * logic in `FileFormatWriter` to add duplicate conversions because the logic there checks the + * partition column using the original plan's output. When the plan is modified with additional + * projections, the partition column check won't match and will not add more conversion. + * + * @param plan The original SparkPlan. + * @param partCols The partition columns. + * @param constraints The defined constraints. + * @return A SparkPlan potentially modified with an additional projection on top of `plan` + */ + override def convertEmptyToNullIfNeeded( + plan: SparkPlan, + partCols: Seq[Attribute], + constraints: Seq[Constraint]): SparkPlan = { + // Reuse the CPU implementation if the plan ends up on the CPU, otherwise do the + // equivalent on the GPU. + plan match { + case g: GpuExec => gpuConvertEmptyToNullIfNeeded(g, partCols, constraints) + case _ => super.convertEmptyToNullIfNeeded(plan, partCols, constraints) + } + } + + override def writeFiles( + inputData: Dataset[_], + additionalConstraints: Seq[Constraint]): Seq[FileAction] = { + writeFiles(inputData, None, additionalConstraints) + } + + protected def applyOptimizeWriteIfNeeded( + spark: SparkSession, + physicalPlan: SparkPlan, + partitionSchema: StructType, + isOptimize: Boolean): SparkPlan = { + val optimizeWriteEnabled = !isOptimize && + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_WRITE_ENABLED) + .orElse(DeltaConfigs.OPTIMIZE_WRITE.fromMetaData(metadata)).getOrElse(false) + if (optimizeWriteEnabled) { + val planWithoutTopRepartition = + DeltaShufflePartitionsUtil.removeTopRepartition(physicalPlan) + val partitioning = DeltaShufflePartitionsUtil.partitioningForRebalance( + physicalPlan.output, partitionSchema, spark.sessionState.conf.numShufflePartitions) + planWithoutTopRepartition match { + case p: GpuExec => + val partMeta = GpuOverrides.wrapPart(partitioning, rapidsConf, None) + partMeta.tagForGpu() + if (partMeta.canThisBeReplaced) { + val plan = GpuOptimizeWriteExchangeExec(partMeta.convertToGpu(), p) + if (GpuShuffleEnv.useGPUShuffle(rapidsConf)) { + GpuCoalesceBatches(plan, TargetSize(rapidsConf.gpuTargetBatchSizeBytes)) + } else { + GpuShuffleCoalesceExec(plan, rapidsConf.gpuTargetBatchSizeBytes) + } + } else { + GpuColumnarToRowExec(OptimizeWriteExchangeExec(partitioning, p)) + } + case p => + OptimizeWriteExchangeExec(partitioning, p) + } + } else { + physicalPlan + } + } + + protected def isOptimizeCommand(plan: LogicalPlan): Boolean = { + val leaves = plan.collectLeaves() + leaves.size == 1 && leaves.head.collect { + case LogicalRelation(HadoopFsRelation( + index: TahoeBatchFileIndex, _, _, _, _, _), _, _, _) => + index.actionType.equals("Optimize") + }.headOption.getOrElse(false) + } + + protected def convertToCpu(plan: SparkPlan): SparkPlan = plan match { + case GpuRowToColumnarExec(p, _) => p + case p: GpuExec => GpuColumnarToRowExec(p) + case p => p + } + + protected def convertToGpu(plan: SparkPlan): SparkPlan = plan match { + case GpuColumnarToRowExec(p, _) => p + case p: GpuExec => p + case p => GpuRowToColumnarExec(p, TargetSize(rapidsConf.gpuTargetBatchSizeBytes)) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimizeExecutor.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimizeExecutor.scala new file mode 100644 index 00000000000..de1e172fd60 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimizeExecutor.scala @@ -0,0 +1,420 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from: + * 1. DoAutoCompaction.scala from PR#1156 at https://github.com/delta-io/delta/pull/1156, + * 2. OptimizeTableCommand.scala from the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import java.util.ConcurrentModificationException + +import scala.annotation.tailrec +import scala.collection.mutable.ArrayBuffer + +import com.databricks.sql.io.skipping.MultiDimClustering +import com.databricks.sql.io.skipping.liquid.{ClusteredTableUtils, ClusteringColumnInfo} +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.DeltaOperations.Operation +import com.databricks.sql.transaction.tahoe.actions.{Action, AddFile, FileAction, RemoveFile} +import com.databricks.sql.transaction.tahoe.commands.DeltaCommand +import com.databricks.sql.transaction.tahoe.commands.optimize._ +import com.databricks.sql.transaction.tahoe.files.SQLMetricsReporting +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.nvidia.spark.rapids.delta.RapidsDeltaSQLConf + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext.SPARK_JOB_GROUP_ID +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric +import org.apache.spark.util.ThreadUtils + +class GpuOptimizeExecutor( + sparkSession: SparkSession, + txn: OptimisticTransaction, + partitionPredicate: Seq[Expression], + zOrderByColumns: Seq[String], + prevCommitActions: Seq[Action]) + extends DeltaCommand with SQLMetricsReporting with Serializable { + + /** Timestamp to use in [[FileAction]] */ + private val operationTimestamp = System.currentTimeMillis + + private val isMultiDimClustering = zOrderByColumns.nonEmpty + private val isAutoCompact = prevCommitActions.nonEmpty + private val optimizeType = GpuOptimizeType(isMultiDimClustering, isAutoCompact) + + def optimize(): Seq[Row] = { + recordDeltaOperation(txn.deltaLog, "delta.optimize") { + val maxFileSize = optimizeType.maxFileSize + require(maxFileSize > 0, "maxFileSize must be > 0") + + val minNumFilesInDir = optimizeType.minNumFiles + val (candidateFiles, filesToProcess) = optimizeType.targetFiles + val partitionSchema = txn.metadata.partitionSchema + + // select all files in case of multi-dimensional clustering + val partitionsToCompact = filesToProcess + .groupBy(_.partitionValues) + .filter { case (_, filesInPartition) => filesInPartition.size >= minNumFilesInDir } + .toSeq + + val groupedJobs = groupFilesIntoBins(partitionsToCompact, maxFileSize) + val jobs = optimizeType.targetBins(groupedJobs) + + val maxThreads = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_THREADS) + val updates = ThreadUtils.parmap(jobs, "OptimizeJob", maxThreads) { partitionBinGroup => + runOptimizeBinJob(txn, partitionBinGroup._1, partitionBinGroup._2, maxFileSize) + }.flatten + + val addedFiles = updates.collect { case a: AddFile => a } + val removedFiles = updates.collect { case r: RemoveFile => r } + if (addedFiles.nonEmpty) { + val operation = DeltaOperations.Optimize(partitionPredicate, zOrderByColumns) + val metrics = createMetrics(sparkSession.sparkContext, addedFiles, removedFiles) + commitAndRetry(txn, operation, updates, metrics) { newTxn => + val newPartitionSchema = newTxn.metadata.partitionSchema + val candidateSetOld = candidateFiles.map(_.path).toSet + val candidateSetNew = newTxn.filterFiles(partitionPredicate).map(_.path).toSet + + // As long as all of the files that we compacted are still part of the table, + // and the partitioning has not changed it is valid to continue to try + // and commit this checkpoint. + if (candidateSetOld.subsetOf(candidateSetNew) && partitionSchema == newPartitionSchema) { + true + } else { + val deleted = candidateSetOld -- candidateSetNew + logWarning(s"The following compacted files were delete " + + s"during checkpoint ${deleted.mkString(",")}. Aborting the compaction.") + false + } + } + } + + val optimizeStats = OptimizeStats() + optimizeStats.addedFilesSizeStats.merge(addedFiles) + optimizeStats.removedFilesSizeStats.merge(removedFiles) + optimizeStats.numPartitionsOptimized = jobs.map(j => j._1).distinct.size + optimizeStats.numBatches = jobs.size + optimizeStats.totalConsideredFiles = candidateFiles.size + optimizeStats.totalFilesSkipped = optimizeStats.totalConsideredFiles - removedFiles.size + optimizeStats.totalClusterParallelism = sparkSession.sparkContext.defaultParallelism + + if (isMultiDimClustering) { + val inputFileStats = + ZOrderFileStats(removedFiles.size, removedFiles.map(_.size.getOrElse(0L)).sum) + optimizeStats.zOrderStats = Some(ZOrderStats( + strategyName = "all", // means process all files in a partition + inputCubeFiles = ZOrderFileStats(0, 0), + inputOtherFiles = inputFileStats, + inputNumCubes = 0, + mergedFiles = inputFileStats, + // There will one z-cube for each partition + numOutputCubes = optimizeStats.numPartitionsOptimized)) + } + + return Seq(Row(txn.deltaLog.dataPath.toString, optimizeStats.toOptimizeMetrics)) + } + } + + /** + * Utility methods to group files into bins for optimize. + * + * @param partitionsToCompact List of files to compact group by partition. + * Partition is defined by the partition values (partCol -> partValue) + * @param maxTargetFileSize Max size (in bytes) of the compaction output file. + * @return Sequence of bins. Each bin contains one or more files from the same + * partition and targeted for one output file. + */ + private def groupFilesIntoBins( + partitionsToCompact: Seq[(Map[String, String], Seq[AddFile])], + maxTargetFileSize: Long): Seq[(Map[String, String], Seq[AddFile])] = { + + partitionsToCompact.flatMap { + case (partition, files) => + val bins = new ArrayBuffer[Seq[AddFile]]() + + val currentBin = new ArrayBuffer[AddFile]() + var currentBinSize = 0L + + files.sortBy(_.size).foreach { file => + // Generally, a bin is a group of existing files, whose total size does not exceed the + // desired maxFileSize. They will be coalesced into a single output file. + // However, if isMultiDimClustering = true, all files in a partition will be read by the + // same job, the data will be range-partitioned and numFiles = totalFileSize / maxFileSize + // will be produced. See below. + if (file.size + currentBinSize > maxTargetFileSize && !isMultiDimClustering) { + bins += currentBin.toVector + currentBin.clear() + currentBin += file + currentBinSize = file.size + } else { + currentBin += file + currentBinSize += file.size + } + } + + if (currentBin.nonEmpty) { + bins += currentBin.toVector + } + + bins.map(b => (partition, b)) + // select bins that have at least two files or in case of multi-dim clustering + // select all bins + .filter(_._2.size > 1 || isMultiDimClustering) + } + } + + private val isClusteredTable = ClusteredTableUtils.isSupported(txn.snapshot.protocol) + + private val clusteringColumns: Seq[String] = { + if (zOrderByColumns.nonEmpty) { + zOrderByColumns + } else if (isClusteredTable) { + ClusteringColumnInfo.extractLogicalNames(txn.snapshot) + } else { + Nil + } + } + + /** + * Utility method to run a Spark job to compact the files in given bin + * + * @param txn [[OptimisticTransaction]] instance in use to commit the changes to DeltaLog. + * @param partition Partition values of the partition that files in [[bin]] belongs to. + * @param bin List of files to compact into one large file. + * @param maxFileSize Targeted output file size in bytes + */ + private def runOptimizeBinJob( + txn: OptimisticTransaction, + partition: Map[String, String], + bin: Seq[AddFile], + maxFileSize: Long): Seq[FileAction] = { + val baseTablePath = txn.deltaLog.dataPath + + val input = txn.deltaLog.createDataFrame(txn.snapshot, bin, actionTypeOpt = Some("Optimize")) + val repartitionDF = if (isMultiDimClustering) { + val totalSize = bin.map(_.size).sum + val approxNumFiles = Math.max(1, totalSize / maxFileSize).toInt + MultiDimClustering.cluster( + input, + approxNumFiles, + clusteringColumns, + "zorder") + } else { + val useRepartition = sparkSession.sessionState.conf.getConf( + DeltaSQLConf.DELTA_OPTIMIZE_REPARTITION_ENABLED) + if (useRepartition) { + input.repartition(numPartitions = 1) + } else { + input.coalesce(numPartitions = 1) + } + } + + val partitionDesc = partition.toSeq.map(entry => entry._1 + "=" + entry._2).mkString(",") + + val partitionName = if (partition.isEmpty) "" else s" in partition ($partitionDesc)" + val description = s"$baseTablePath
Optimizing ${bin.size} files" + partitionName + sparkSession.sparkContext.setJobGroup( + sparkSession.sparkContext.getLocalProperty(SPARK_JOB_GROUP_ID), + description) + + val addFiles = txn.writeFiles(repartitionDF).collect { + case a: AddFile => + a.copy(dataChange = false) + case other => + throw new IllegalStateException( + s"Unexpected action $other with type ${other.getClass}. File compaction job output" + + s"should only have AddFiles") + } + val removeFiles = bin.map(f => f.removeWithTimestamp(operationTimestamp, dataChange = false)) + val updates = addFiles ++ removeFiles + updates + } + + private type PartitionedBin = (Map[String, String], Seq[AddFile]) + + private trait GpuOptimizeType { + def minNumFiles: Long + + def maxFileSize: Long = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE) + + def targetFiles: (Seq[AddFile], Seq[AddFile]) + + def targetBins(jobs: Seq[PartitionedBin]): Seq[PartitionedBin] = jobs + } + + private case class GpuCompaction() extends GpuOptimizeType { + def minNumFiles: Long = 2 + + def targetFiles: (Seq[AddFile], Seq[AddFile]) = { + val minFileSize = sparkSession.sessionState.conf.getConf( + DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE) + require(minFileSize > 0, "minFileSize must be > 0") + val candidateFiles = txn.filterFiles(partitionPredicate) + val filesToProcess = candidateFiles.filter(_.size < minFileSize) + (candidateFiles, filesToProcess) + } + } + + private case class GpuMultiDimOrdering() extends GpuOptimizeType { + def minNumFiles: Long = 1 + + def targetFiles: (Seq[AddFile], Seq[AddFile]) = { + // select all files in case of multi-dimensional clustering + val candidateFiles = txn.filterFiles(partitionPredicate) + (candidateFiles, candidateFiles) + } + } + + private case class GpuAutoCompaction() extends GpuOptimizeType { + def minNumFiles: Long = { + val minNumFiles = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_AUTO_COMPACT_MIN_NUM_FILES) + require(minNumFiles > 0, "minNumFiles must be > 0") + minNumFiles + } + + override def maxFileSize: Long = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_AUTO_COMPACT_MAX_FILE_SIZE) + .getOrElse(128 * 1024 * 1024) + + override def targetFiles: (Seq[AddFile], Seq[AddFile]) = { + val autoCompactTarget = + sparkSession.sessionState.conf.getConf(RapidsDeltaSQLConf.AUTO_COMPACT_TARGET) + // Filter the candidate files according to autoCompact.target config. + lazy val addedFiles = prevCommitActions.collect { case a: AddFile => a } + val candidateFiles = autoCompactTarget match { + case "table" => + txn.filterFiles() + case "commit" => + addedFiles + case "partition" => + val eligiblePartitions = addedFiles.map(_.partitionValues).toSet + txn.filterFiles().filter(f => eligiblePartitions.contains(f.partitionValues)) + case _ => + logError(s"Invalid config for autoCompact.target: $autoCompactTarget. " + + s"Falling back to the default value 'table'.") + txn.filterFiles() + } + val filesToProcess = candidateFiles.filter(_.size < maxFileSize) + (candidateFiles, filesToProcess) + } + + override def targetBins(jobs: Seq[PartitionedBin]): Seq[PartitionedBin] = { + var acc = 0L + val maxCompactBytes = + sparkSession.sessionState.conf.getConf(RapidsDeltaSQLConf.AUTO_COMPACT_MAX_COMPACT_BYTES) + // bins with more files are prior to less files. + jobs + .sortBy { case (_, filesInBin) => -filesInBin.length } + .takeWhile { case (_, filesInBin) => + acc += filesInBin.map(_.size).sum + acc <= maxCompactBytes + } + } + } + + private object GpuOptimizeType { + + def apply(isMultiDimClustering: Boolean, isAutoCompact: Boolean): GpuOptimizeType = { + if (isMultiDimClustering) { + GpuMultiDimOrdering() + } else if (isAutoCompact) { + GpuAutoCompaction() + } else { + GpuCompaction() + } + } + } + + /** + * Attempts to commit the given actions to the log. In the case of a concurrent update, + * the given function will be invoked with a new transaction to allow custom conflict + * detection logic to indicate it is safe to try again, by returning `true`. + * + * This function will continue to try to commit to the log as long as `f` returns `true`, + * otherwise throws a subclass of [[ConcurrentModificationException]]. + */ + @tailrec + private def commitAndRetry( + txn: OptimisticTransaction, + optimizeOperation: Operation, + actions: Seq[Action], + metrics: Map[String, SQLMetric])(f: OptimisticTransaction => Boolean) + : Unit = { + try { + txn.registerSQLMetrics(sparkSession, metrics) + txn.commit(actions, optimizeOperation) + } catch { + case e: ConcurrentModificationException => + val newTxn = txn.deltaLog.startTransaction(Option.empty[CatalogTable]) + if (f(newTxn)) { + logInfo("Retrying commit after checking for semantic conflicts with concurrent updates.") + commitAndRetry(newTxn, optimizeOperation, actions, metrics)(f) + } else { + logWarning("Semantic conflicts detected. Aborting operation.") + throw e + } + } + } + + /** Create a map of SQL metrics for adding to the commit history. */ + private def createMetrics( + sparkContext: SparkContext, + addedFiles: Seq[AddFile], + removedFiles: Seq[RemoveFile]): Map[String, SQLMetric] = { + + def setAndReturnMetric(description: String, value: Long) = { + val metric = createMetric(sparkContext, description) + metric.set(value) + metric + } + + def totalSize(actions: Seq[FileAction]): Long = { + var totalSize = 0L + actions.foreach { file => + val fileSize = file match { + case addFile: AddFile => addFile.size + case removeFile: RemoveFile => removeFile.size.getOrElse(0L) + case default => + throw new IllegalArgumentException(s"Unknown FileAction type: ${default.getClass}") + } + totalSize += fileSize + } + totalSize + } + + val sizeStats = FileSizeStatsWithHistogram.create(addedFiles.map(_.size).sorted) + Map[String, SQLMetric]( + "minFileSize" -> setAndReturnMetric("minimum file size", sizeStats.get.min), + "p25FileSize" -> setAndReturnMetric("25th percentile file size", sizeStats.get.p25), + "p50FileSize" -> setAndReturnMetric("50th percentile file size", sizeStats.get.p50), + "p75FileSize" -> setAndReturnMetric("75th percentile file size", sizeStats.get.p75), + "maxFileSize" -> setAndReturnMetric("maximum file size", sizeStats.get.max), + "numAddedFiles" -> setAndReturnMetric("total number of files added.", addedFiles.size), + "numRemovedFiles" -> setAndReturnMetric("total number of files removed.", removedFiles.size), + "numAddedBytes" -> setAndReturnMetric("total number of bytes added", totalSize(addedFiles)), + "numRemovedBytes" -> + setAndReturnMetric("total number of bytes removed", totalSize(removedFiles))) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuUpdateCommand.scala b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuUpdateCommand.scala new file mode 100644 index 00000000000..c7e69eede25 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuUpdateCommand.scala @@ -0,0 +1,282 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * This file was derived from UpdateCommand.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * 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.databricks.sql.transaction.tahoe.rapids + +import com.databricks.sql.transaction.tahoe.{DeltaLog, DeltaOperations, DeltaTableUtils, DeltaUDF, OptimisticTransaction} +import com.databricks.sql.transaction.tahoe.DeltaCommitTag._ +import com.databricks.sql.transaction.tahoe.RowTracking +import com.databricks.sql.transaction.tahoe.actions.{AddCDCFile, AddFile, FileAction} +import com.databricks.sql.transaction.tahoe.commands.{DeltaCommand, DMLUtils, UpdateCommand, UpdateMetric} +import com.databricks.sql.transaction.tahoe.files.{TahoeBatchFileIndex, TahoeFileIndex} +import com.nvidia.spark.rapids.delta.GpuDeltaMetricUpdateUDF +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkContext +import org.apache.spark.sql.{Column, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Literal} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetrics.{createMetric, createTimingMetric} +import org.apache.spark.sql.functions.input_file_name +import org.apache.spark.sql.types.LongType + +case class GpuUpdateCommand( + gpuDeltaLog: GpuDeltaLog, + tahoeFileIndex: TahoeFileIndex, + target: LogicalPlan, + updateExpressions: Seq[Expression], + condition: Option[Expression]) + extends LeafRunnableCommand with DeltaCommand { + + override val output: Seq[Attribute] = { + Seq(AttributeReference("num_affected_rows", LongType)()) + } + + override def innerChildren: Seq[QueryPlan[_]] = Seq(target) + + @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() + + override lazy val metrics = Map[String, SQLMetric]( + "numAddedFiles" -> createMetric(sc, "number of files added."), + "numRemovedFiles" -> createMetric(sc, "number of files removed."), + "numUpdatedRows" -> createMetric(sc, "number of rows updated."), + "numCopiedRows" -> createMetric(sc, "number of rows copied."), + "executionTimeMs" -> + createTimingMetric(sc, "time taken to execute the entire operation"), + "scanTimeMs" -> + createTimingMetric(sc, "time taken to scan the files for matches"), + "rewriteTimeMs" -> + createTimingMetric(sc, "time taken to rewrite the matched files"), + "numAddedChangeFiles" -> createMetric(sc, "number of change data capture files generated"), + "changeFileBytes" -> createMetric(sc, "total size of change data capture files generated"), + "numTouchedRows" -> createMetric(sc, "number of rows touched (copied + updated)"), + "numDeletionVectorsAdded" -> createMetric(sc, "number of deletion vectors added."), + "numDeletionVectorsRemoved" -> createMetric(sc, "number of deletion vectors removed."), + "numDeletionVectorsUpdated" -> createMetric(sc, "number of deletion vectors updated.") + ) + + final override def run(sparkSession: SparkSession): Seq[Row] = { + recordDeltaOperation(tahoeFileIndex.deltaLog, "delta.dml.update") { + val deltaLog = tahoeFileIndex.deltaLog + gpuDeltaLog.withNewTransaction { txn => + DeltaLog.assertRemovable(txn.snapshot) + performUpdate(sparkSession, deltaLog, txn) + } + // Re-cache all cached plans(including this relation itself, if it's cached) that refer to + // this data source relation. + sparkSession.sharedState.cacheManager.recacheByPlan(sparkSession, target) + } + Seq(Row(metrics("numUpdatedRows").value)) + } + + private def performUpdate( + sparkSession: SparkSession, deltaLog: DeltaLog, txn: OptimisticTransaction): Unit = { + import com.databricks.sql.transaction.tahoe.implicits._ + + var numTouchedFiles: Long = 0 + var numRewrittenFiles: Long = 0 + var numAddedChangeFiles: Long = 0 + var changeFileBytes: Long = 0 + var scanTimeMs: Long = 0 + var rewriteTimeMs: Long = 0 + + val startTime = System.nanoTime() + val numFilesTotal = txn.snapshot.numOfFiles + + val updateCondition = condition.getOrElse(Literal.TrueLiteral) + val (metadataPredicates, dataPredicates) = + DeltaTableUtils.splitMetadataAndDataPredicates( + updateCondition, txn.metadata.partitionColumns, sparkSession) + val candidateFiles = txn.filterFiles(metadataPredicates ++ dataPredicates) + val nameToAddFile = generateCandidateFileMap(deltaLog.dataPath, candidateFiles) + + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + + val filesToRewrite: Seq[AddFile] = if (candidateFiles.isEmpty) { + // Case 1: Do nothing if no row qualifies the partition predicates + // that are part of Update condition + Nil + } else if (dataPredicates.isEmpty) { + // Case 2: Update all the rows from the files that are in the specified partitions + // when the data filter is empty + candidateFiles + } else { + // Case 3: Find all the affected files using the user-specified condition + val fileIndex = new TahoeBatchFileIndex( + sparkSession, "update", candidateFiles, deltaLog, tahoeFileIndex.path, txn.snapshot) + // Keep everything from the resolved target except a new TahoeFileIndex + // that only involves the affected files instead of all files. + val newTarget = DeltaTableUtils.replaceFileIndex(target, fileIndex) + val data = Dataset.ofRows(sparkSession, newTarget) + val updatedRowCount = metrics("numUpdatedRows") + val updatedRowUdf = DeltaUDF.boolean { + new GpuDeltaMetricUpdateUDF(updatedRowCount) + }.asNondeterministic() + val pathsToRewrite = + withStatusCode("DELTA", UpdateCommand.FINDING_TOUCHED_FILES_MSG) { + data.filter(new Column(updateCondition)) + .select(input_file_name()) + .filter(updatedRowUdf()) + .distinct() + .as[String] + .collect() + } + + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + + pathsToRewrite.map(getTouchedFile(deltaLog.dataPath, _, nameToAddFile)).toSeq + } + + numTouchedFiles = filesToRewrite.length + + val newActions = if (filesToRewrite.isEmpty) { + // Do nothing if no row qualifies the UPDATE condition + Nil + } else { + // Generate the new files containing the updated values + withStatusCode("DELTA", UpdateCommand.rewritingFilesMsg(filesToRewrite.size)) { + rewriteFiles(sparkSession, txn, tahoeFileIndex.path, + filesToRewrite.map(_.path), nameToAddFile, updateCondition) + } + } + + rewriteTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 - scanTimeMs + + val (changeActions, addActions) = newActions.partition(_.isInstanceOf[AddCDCFile]) + numRewrittenFiles = addActions.size + numAddedChangeFiles = changeActions.size + changeFileBytes = changeActions.collect { case f: AddCDCFile => f.size }.sum + + val totalActions = if (filesToRewrite.isEmpty) { + // Do nothing if no row qualifies the UPDATE condition + Nil + } else { + // Delete the old files and return those delete actions along with the new AddFile actions for + // files containing the updated values + val operationTimestamp = System.currentTimeMillis() + val deleteActions = filesToRewrite.map(_.removeWithTimestamp(operationTimestamp)) + + deleteActions ++ newActions + } + + if (totalActions.nonEmpty) { + metrics("numAddedFiles").set(numRewrittenFiles) + metrics("numAddedChangeFiles").set(numAddedChangeFiles) + metrics("changeFileBytes").set(changeFileBytes) + metrics("numRemovedFiles").set(numTouchedFiles) + metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) + metrics("scanTimeMs").set(scanTimeMs) + metrics("rewriteTimeMs").set(rewriteTimeMs) + // In the case where the numUpdatedRows is not captured, we can siphon out the metrics from + // the BasicWriteStatsTracker. This is for case 2 where the update condition contains only + // metadata predicates and so the entire partition is re-written. + val outputRows = txn.getMetric("numOutputRows").map(_.value).getOrElse(-1L) + if (metrics("numUpdatedRows").value == 0 && outputRows != 0 && + metrics("numCopiedRows").value == 0) { + // We know that numTouchedRows = numCopiedRows + numUpdatedRows. + // Since an entire partition was re-written, no rows were copied. + // So numTouchedRows == numUpdateRows + metrics("numUpdatedRows").set(metrics("numTouchedRows").value) + } else { + // This is for case 3 where the update condition contains both metadata and data predicates + // so relevant files will have some rows updated and some rows copied. We don't need to + // consider case 1 here, where no files match the update condition, as we know that + // `totalActions` is empty. + metrics("numCopiedRows").set( + metrics("numTouchedRows").value - metrics("numUpdatedRows").value) + } + metrics("numDeletionVectorsAdded").set(0) + metrics("numDeletionVectorsRemoved").set(0) + metrics("numDeletionVectorsUpdated").set(0) + txn.registerSQLMetrics(sparkSession, metrics) + val tags = DMLUtils.TaggedCommitData.EMPTY + .withTag(PreservedRowTrackingTag, RowTracking.isEnabled(txn.protocol, txn.metadata)) + .withTag(NoRowsCopiedTag, metrics("numCopiedRows").value == 0) + txn.commitIfNeeded(totalActions, DeltaOperations.Update(condition), tags.stringTags) + // This is needed to make the SQL metrics visible in the Spark UI + val executionId = sparkSession.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates( + sparkSession.sparkContext, executionId, metrics.values.toSeq) + } + + recordDeltaEvent( + deltaLog, + "delta.dml.update.stats", + data = UpdateMetric( + condition = condition.map(_.sql).getOrElse("true"), + numFilesTotal, + numTouchedFiles, + numRewrittenFiles, + numAddedChangeFiles, + changeFileBytes, + scanTimeMs, + rewriteTimeMs, + // We don't support deletion vectors + numDeletionVectorsAdded = 0, + numDeletionVectorsRemoved = 0, + numDeletionVectorsUpdated = 0) + ) + } + + /** + * Scan all the affected files and write out the updated files. + * + * When CDF is enabled, includes the generation of CDC preimage and postimage columns for + * changed rows. + * + * @return the list of [[AddFile]]s and [[AddCDCFile]]s that have been written. + */ + private def rewriteFiles( + spark: SparkSession, + txn: OptimisticTransaction, + rootPath: Path, + inputLeafFiles: Seq[String], + nameToAddFileMap: Map[String, AddFile], + condition: Expression): Seq[FileAction] = { + // Containing the map from the relative file path to AddFile + val baseRelation = buildBaseRelation( + spark, txn, "update", rootPath, inputLeafFiles, nameToAddFileMap) + val newTarget = DeltaTableUtils.replaceFileIndex(target, baseRelation.location) + val targetDf = Dataset.ofRows(spark, newTarget) + + // Number of total rows that we have seen, i.e. are either copying or updating (sum of both). + // This will be used later, along with numUpdatedRows, to determine numCopiedRows. + val numTouchedRows = metrics("numTouchedRows") + val numTouchedRowsUdf = DeltaUDF.boolean { + new GpuDeltaMetricUpdateUDF(numTouchedRows) + }.asNondeterministic() + + val updatedDataFrame = UpdateCommand.withUpdatedColumns( + target.output, + updateExpressions, + condition, + targetDf + .filter(numTouchedRowsUdf()) + .withColumn(UpdateCommand.CONDITION_COLUMN_NAME, new Column(condition)), + UpdateCommand.shouldOutputCdc(txn)) + + txn.writeFiles(updatedDataFrame) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProbe.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProbe.scala new file mode 100644 index 00000000000..32b7656b45b --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProbe.scala @@ -0,0 +1,25 @@ +/* + * Copyright (c) 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.delta + +/** + * Implements the Delta Probe interface for probing the Delta Lake provider on Databricks. + * @note This is instantiated via reflection from ShimLoader. + */ +class DeltaProbeImpl extends DeltaProbe { + // Delta Lake is built-in for Databricks instances, so no probing is necessary. + override def getDeltaProvider: DeltaProvider = DeltaSpark341DBProvider +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/DeltaSpark341DBProvider.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/DeltaSpark341DBProvider.scala new file mode 100644 index 00000000000..cd204fa0440 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/DeltaSpark341DBProvider.scala @@ -0,0 +1,55 @@ +/* + * Copyright (c) 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.delta + +import com.databricks.sql.transaction.tahoe.rapids.GpuDeltaCatalog +import com.nvidia.spark.rapids.{AtomicCreateTableAsSelectExecMeta, AtomicReplaceTableAsSelectExecMeta, GpuExec} + +import org.apache.spark.sql.execution.datasources.v2.{AtomicCreateTableAsSelectExec, AtomicReplaceTableAsSelectExec} +import org.apache.spark.sql.execution.datasources.v2.rapids.{GpuAtomicCreateTableAsSelectExec, GpuAtomicReplaceTableAsSelectExec} + +object DeltaSpark341DBProvider extends DatabricksDeltaProviderBase { + + override def convertToGpu( + cpuExec: AtomicCreateTableAsSelectExec, + meta: AtomicCreateTableAsSelectExecMeta): GpuExec = { + GpuAtomicCreateTableAsSelectExec( + cpuExec.output, + new GpuDeltaCatalog(cpuExec.catalog, meta.conf), + cpuExec.ident, + cpuExec.partitioning, + cpuExec.query, + cpuExec.tableSpec, + cpuExec.writeOptions, + cpuExec.ifNotExists) + } + + override def convertToGpu( + cpuExec: AtomicReplaceTableAsSelectExec, + meta: AtomicReplaceTableAsSelectExecMeta): GpuExec = { + GpuAtomicReplaceTableAsSelectExec( + cpuExec.output, + new GpuDeltaCatalog(cpuExec.catalog, meta.conf), + cpuExec.ident, + cpuExec.partitioning, + cpuExec.query, + cpuExec.tableSpec, + cpuExec.writeOptions, + cpuExec.orCreate, + cpuExec.invalidateCache) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..088a2a788da --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,125 @@ +/* + * Copyright (c) 2023-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 com.nvidia.spark.rapids.delta + +import java.net.URI + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat, IdMapping} +import com.databricks.sql.transaction.tahoe.DeltaParquetFileFormat.{DeletionVectorDescriptorWithFilterType, IS_ROW_DELETED_COLUMN_NAME} +import com.nvidia.spark.rapids.{GpuMetric, RapidsConf, SparkPlanMeta} +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormatUtils.addMetadataColumnToIterator +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +case class GpuDeltaParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType, + isSplittable: Boolean, + disablePushDown: Boolean, + broadcastDvMap: Option[Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]]] +) extends GpuDeltaParquetFileFormatBase { + + if (columnMappingMode == IdMapping) { + val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)), + s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode") + val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)), + s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode") + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = isSplittable + + override def buildReaderWithPartitionValuesAndMetrics( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration, + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { + + val dataReader = super.buildReaderWithPartitionValuesAndMetrics( + sparkSession, + dataSchema, + partitionSchema, + requiredSchema, + filters, + options, + hadoopConf, + metrics, + alluxioPathReplacementMap) + + val delVecs = broadcastDvMap + val maxDelVecScatterBatchSize = RapidsConf + .DELTA_LOW_SHUFFLE_MERGE_SCATTER_DEL_VECTOR_BATCH_SIZE + .get(sparkSession.sessionState.conf) + + val delVecScatterTimeMetric = metrics(GpuMetric.DELETION_VECTOR_SCATTER_TIME) + val delVecSizeMetric = metrics(GpuMetric.DELETION_VECTOR_SIZE) + + (file: PartitionedFile) => { + val input = dataReader(file) + val dv = delVecs.flatMap(_.value.get(new URI(file.filePath.toString()))) + .map { dv => + delVecSizeMetric += dv.descriptor.inlineData.length + RoaringBitmapWrapper.deserializeFromBytes(dv.descriptor.inlineData).inner + } + addMetadataColumnToIterator(prepareSchema(requiredSchema), + dv, + input.asInstanceOf[Iterator[ColumnarBatch]], + maxDelVecScatterBatchSize, + delVecScatterTimeMetric + ).asInstanceOf[Iterator[InternalRow]] + } + } +} + +object GpuDeltaParquetFileFormat { + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat.asInstanceOf[DeltaParquetFileFormat] + val requiredSchema = meta.wrapped.requiredSchema + if (requiredSchema.exists(_.name == IS_ROW_DELETED_COLUMN_NAME)) { + meta.willNotWorkOnGpu( + s"reading metadata column $IS_ROW_DELETED_COLUMN_NAME is not supported") + } + if (format.hasDeletionVectorMap) { + meta.willNotWorkOnGpu("deletion vectors are not supported") + } + } + + def convertToGpu(fmt: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { + GpuDeltaParquetFileFormat(fmt.columnMappingMode, fmt.referenceSchema, fmt.isSplittable, + fmt.disablePushDowns, fmt.broadcastDvMap) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeleteCommandMetaShim.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeleteCommandMetaShim.scala new file mode 100644 index 00000000000..96863c71ad0 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeleteCommandMetaShim.scala @@ -0,0 +1,43 @@ +/* + * Copyright (c) 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.delta.shims + +import com.databricks.sql.transaction.tahoe.commands.DeletionVectorUtils +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.nvidia.spark.rapids.delta.{DeleteCommandEdgeMeta, DeleteCommandMeta} + +object DeleteCommandMetaShim { + def tagForGpu(meta: DeleteCommandMeta): Unit = { + val dvFeatureEnabled = DeletionVectorUtils.deletionVectorsWritable( + meta.deleteCmd.deltaLog.unsafeVolatileSnapshot) + if (dvFeatureEnabled && meta.deleteCmd.conf.getConf( + DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS)) { + // https://github.com/NVIDIA/spark-rapids/issues/8654 + meta.willNotWorkOnGpu("Deletion vector writes are not supported on GPU") + } + } + + def tagForGpu(meta: DeleteCommandEdgeMeta): Unit = { + val dvFeatureEnabled = DeletionVectorUtils.deletionVectorsWritable( + meta.deleteCmd.deltaLog.unsafeVolatileSnapshot) + if (dvFeatureEnabled && meta.deleteCmd.conf.getConf( + DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS)) { + // https://github.com/NVIDIA/spark-rapids/issues/8654 + meta.willNotWorkOnGpu("Deletion vector writes are not supported on GPU") + } + } +} \ No newline at end of file diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala new file mode 100644 index 00000000000..0bd231e05a6 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala @@ -0,0 +1,32 @@ +/* + * Copyright (c) 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.delta.shims + +import com.databricks.sql.transaction.tahoe.DeltaLog +import com.databricks.sql.transaction.tahoe.actions.Metadata + +import org.apache.spark.sql.execution.datasources.FileFormat + +object DeltaLogShim { + def fileFormat(deltaLog: DeltaLog): FileFormat = { + deltaLog.fileFormat(deltaLog.unsafeVolatileSnapshot.protocol, + deltaLog.unsafeVolatileSnapshot.metadata) + } + def getMetadata(deltaLog: DeltaLog): Metadata = { + deltaLog.unsafeVolatileSnapshot.metadata + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/InvariantViolationExceptionShim.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/InvariantViolationExceptionShim.scala new file mode 100644 index 00000000000..58714a91fd4 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/InvariantViolationExceptionShim.scala @@ -0,0 +1,30 @@ +/* + * Copyright (c) 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.delta.shims + +import com.databricks.sql.transaction.tahoe.constraints.Constraints._ +import com.databricks.sql.transaction.tahoe.schema.DeltaInvariantViolationException +import com.databricks.sql.transaction.tahoe.schema.InvariantViolationException + +object InvariantViolationExceptionShim { + def apply(c: Check, m: Map[String, Any]): InvariantViolationException = { + DeltaInvariantViolationException(c, m) + } + + def apply(c: NotNull): InvariantViolationException = { + DeltaInvariantViolationException(c) + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MergeIntoCommandMetaShim.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MergeIntoCommandMetaShim.scala new file mode 100644 index 00000000000..5a2b4e7b52e --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MergeIntoCommandMetaShim.scala @@ -0,0 +1,121 @@ +/* + * Copyright (c) 2023-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 com.nvidia.spark.rapids.delta.shims + +import com.databricks.sql.transaction.tahoe.commands.{MergeIntoCommand, MergeIntoCommandEdge} +import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaLog, GpuLowShuffleMergeCommand, GpuMergeIntoCommand} +import com.nvidia.spark.rapids.{RapidsConf, RapidsReaderType} +import com.nvidia.spark.rapids.delta.{MergeIntoCommandEdgeMeta, MergeIntoCommandMeta} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.command.RunnableCommand + +object MergeIntoCommandMetaShim extends Logging { + def tagForGpu(meta: MergeIntoCommandMeta, mergeCmd: MergeIntoCommand): Unit = { + // see https://github.com/NVIDIA/spark-rapids/issues/8415 for more information + if (mergeCmd.notMatchedBySourceClauses.nonEmpty) { + meta.willNotWorkOnGpu("notMatchedBySourceClauses not supported on GPU") + } + } + + def tagForGpu(meta: MergeIntoCommandEdgeMeta, mergeCmd: MergeIntoCommandEdge): Unit = { + // see https://github.com/NVIDIA/spark-rapids/issues/8415 for more information + if (mergeCmd.notMatchedBySourceClauses.nonEmpty) { + meta.willNotWorkOnGpu("notMatchedBySourceClauses not supported on GPU") + } + } + + def convertToGpu(mergeCmd: MergeIntoCommand, conf: RapidsConf): RunnableCommand = { + // TODO: Currently we only support low shuffler merge only when parquet per file read is enabled + // due to the limitation of implementing row index metadata column. + if (conf.isDeltaLowShuffleMergeEnabled) { + if (conf.isParquetPerFileReadEnabled) { + GpuLowShuffleMergeCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } else { + logWarning(s"""Low shuffle merge disabled since ${RapidsConf.PARQUET_READER_TYPE} is + not set to ${RapidsReaderType.PERFILE}. Falling back to classic merge.""") + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } + } else { + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } + } + + def convertToGpu(mergeCmd: MergeIntoCommandEdge, conf: RapidsConf): RunnableCommand = { + // TODO: Currently we only support low shuffler merge only when parquet per file read is enabled + // due to the limitation of implementing row index metadata column. + if (conf.isDeltaLowShuffleMergeEnabled) { + if (conf.isParquetPerFileReadEnabled) { + GpuLowShuffleMergeCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } else { + logWarning(s"""Low shuffle merge is still disable since ${RapidsConf.PARQUET_READER_TYPE} is + not set to ${RapidsReaderType.PERFILE}. Falling back to classic merge.""") + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } + } else { + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } + } +} diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MetadataShims.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MetadataShims.scala new file mode 100644 index 00000000000..8f5196d7c66 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MetadataShims.scala @@ -0,0 +1,25 @@ +/* + * Copyright (c) 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.delta.shims + +import com.databricks.sql.transaction.tahoe.stats.DeltaStatistics + +trait ShimUsesMetadataFields { + val NUM_RECORDS = DeltaStatistics.NUM_RECORDS + val MIN = DeltaStatistics.MIN + val MAX = DeltaStatistics.MAX + val NULL_COUNT = DeltaStatistics.NULL_COUNT +} \ No newline at end of file diff --git a/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/ShimDeltaUDF.scala b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/ShimDeltaUDF.scala new file mode 100644 index 00000000000..fd9052d9691 --- /dev/null +++ b/delta-lake/delta-spark350db/src/main/scala/com/nvidia/spark/rapids/delta/shims/ShimDeltaUDF.scala @@ -0,0 +1,25 @@ +/* + * Copyright (c) 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.delta.shims + +import com.databricks.sql.transaction.tahoe.DeltaUDF + +import org.apache.spark.sql.expressions.UserDefinedFunction + +object ShimDeltaUDF { + def stringStringUdf(f: String => String): UserDefinedFunction = DeltaUDF.stringFromString(f) +} diff --git a/jenkins/Jenkinsfile-blossom.premerge-databricks b/jenkins/Jenkinsfile-blossom.premerge-databricks index 5b0a2bf1226..e1f3f56c67d 100644 --- a/jenkins/Jenkinsfile-blossom.premerge-databricks +++ b/jenkins/Jenkinsfile-blossom.premerge-databricks @@ -91,7 +91,7 @@ pipeline { // 'name' and 'value' only supprt literal string in the declarative Jenkins // Refer to Jenkins issue https://issues.jenkins.io/browse/JENKINS-62127 name 'DB_RUNTIME' - values '11.3', '12.2', '13.3' + values '11.3', '12.2', '13.3', '14.3' } } stages { diff --git a/jenkins/databricks/install_deps.py b/jenkins/databricks/install_deps.py index 7b77396b3f8..88cfc321204 100644 --- a/jenkins/databricks/install_deps.py +++ b/jenkins/databricks/install_deps.py @@ -42,6 +42,9 @@ def define_deps(spark_version, scala_version): elif spark_version.startswith('3.4'): spark_prefix = '----ws_3_4' mvn_prefix = '--mvn' + elif spark_version.startswith('3.5'): + spark_prefix = '----ws_3_5' + mvn_prefix = '--mvn' spark_suffix = f'hive-{hive_version}__hadoop-{hadoop_version}_{scala_version}' @@ -69,7 +72,7 @@ def define_deps(spark_version, scala_version): Artifact('org.apache.spark', f'spark-core_{scala_version}', f'{spark_prefix}--core--core-{spark_suffix}_deploy.jar'), Artifact('org.apache.spark', f'spark-versions_{scala_version}', - f'spark--versions--*--shim_{scala_version}_deploy.jar'), + f'spark--versions--*--shim*_{scala_version}_deploy.jar'), Artifact('org.apache.spark', f'databricks-versions_{scala_version}', f'common--build-info--build-info-spark_*_{scala_version}_deploy.jar'), # Spark Hive Patches @@ -125,7 +128,7 @@ def define_deps(spark_version, scala_version): Artifact('com.fasterxml.jackson.core', 'jackson-annotations', f'{prefix_ws_sp_mvn_hadoop}--com.fasterxml.jackson.core--jackson-annotations--com.fasterxml.jackson.core__jackson-annotations__*.jar'), Artifact('org.apache.spark', f'spark-avro_{scala_version}', - f'{spark_prefix}--vendor--avro--avro-*.jar'), + f'{prefix_ws_sp_mvn_hadoop}--org.apache.avro--avro--org.apache.avro*.jar' if spark_version.startswith('3.5') else f'{spark_prefix}--vendor--avro--avro-*.jar'), Artifact('org.apache.avro', 'avro-mapred', f'{prefix_ws_sp_mvn_hadoop}--org.apache.avro--avro-mapred--org.apache.avro__avro-mapred__*.jar'), Artifact('org.apache.avro', 'avro', @@ -133,7 +136,7 @@ def define_deps(spark_version, scala_version): ] # Parquet - if spark_version.startswith('3.4'): + if spark_version.startswith('3.4') or spark_version.startswith('3.5'): deps += [ Artifact('org.apache.parquet', 'parquet-hadoop', f'{spark_prefix}--third_party--parquet-mr--parquet-hadoop--parquet-hadoop-shaded--*--libparquet-hadoop-internal.jar'), @@ -162,7 +165,7 @@ def define_deps(spark_version, scala_version): # log4j-core - if spark_version.startswith('3.3') or spark_version.startswith('3.4'): + if spark_version.startswith('3.3') or spark_version.startswith('3.4') or spark_version.startswith('3.5'): deps += Artifact('org.apache.logging.log4j', 'log4j-core', f'{prefix_ws_sp_mvn_hadoop}--org.apache.logging.log4j--log4j-core--org.apache.logging.log4j__log4j-core__*.jar'), @@ -172,7 +175,7 @@ def define_deps(spark_version, scala_version): f'{prefix_ws_sp_mvn_hadoop}--org.scala-lang.modules--scala-parser-combinators_{scala_version}-*.jar') ] - if spark_version.startswith('3.4'): + if spark_version.startswith('3.4') or spark_version.startswith('3.5'): deps += [ # Spark Internal Logging Artifact('org.apache.spark', f'spark-common-utils_{scala_version}', f'{spark_prefix}--common--utils--common-utils-hive-2.3__hadoop-3.2_2.12_deploy.jar'), @@ -180,6 +183,12 @@ def define_deps(spark_version, scala_version): Artifact('org.apache.spark', f'spark-sql-api_{scala_version}', f'{spark_prefix}--sql--api--sql-api-hive-2.3__hadoop-3.2_2.12_deploy.jar') ] + if spark_version.startswith('3.5'): + deps += [ + Artifact('org.scala-lang.modules', f'scala-collection-compat_{scala_version}', + f'{prefix_ws_sp_mvn_hadoop}--org.scala-lang.modules--scala-collection-compat_{scala_version}--org.scala-lang.modules__scala-collection-compat_{scala_version}__2.11.0.jar'), + Artifact('org.apache.avro', f'avro-connector', f'{spark_prefix}--connector--avro--avro-hive-2.3__hadoop-3.2_2.12_shaded---606136534--avro-unshaded-hive-2.3__hadoop-3.2_2.12_deploy.jar') + ] return deps diff --git a/pom.xml b/pom.xml index 045dc94b3cb..92a0c7170dd 100644 --- a/pom.xml +++ b/pom.xml @@ -549,6 +549,31 @@ delta-lake/delta-stub
+ + + release350db + + + buildver + 350db + + + + + 3.4.4 + spark350db + ${spark350db.version} + ${spark350db.version} + 3.3.1 + true + 1.12.0 + ${spark330.iceberg.version} + + + shim-deps/databricks + delta-lake/delta-spark350db + + release351 @@ -781,6 +806,7 @@ 3.3.0-databricks 3.3.2-databricks 3.4.1-databricks + 3.5.0-databricks 3.5.0 3.5.1 3.5.2 diff --git a/scala2.13/aggregator/pom.xml b/scala2.13/aggregator/pom.xml index a653525b1f7..dab195a92e7 100644 --- a/scala2.13/aggregator/pom.xml +++ b/scala2.13/aggregator/pom.xml @@ -711,6 +711,23 @@ + + release350db + + + buildver + 350db + + + + + com.nvidia + rapids-4-spark-delta-spark350db_${scala.binary.version} + ${project.version} + ${spark.version.classifier} + + + release351 diff --git a/scala2.13/delta-lake/delta-spark350db/pom.xml b/scala2.13/delta-lake/delta-spark350db/pom.xml new file mode 100644 index 00000000000..ee194daa1ec --- /dev/null +++ b/scala2.13/delta-lake/delta-spark350db/pom.xml @@ -0,0 +1,85 @@ + + + + 4.0.0 + + + com.nvidia + rapids-4-spark-shim-deps-parent_2.13 + 24.10.0-SNAPSHOT + ../../shim-deps/pom.xml + + + rapids-4-spark-delta-spark350db_2.13 + RAPIDS Accelerator for Apache Spark Databricks 13.3 Delta Lake Support + Databricks 13.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark + 24.10.0-SNAPSHOT + + + false + **/* + package + + + + + org.roaringbitmap + RoaringBitmap + + + com.nvidia + rapids-4-spark-sql_${scala.binary.version} + ${project.version} + ${spark.version.classifier} + provided + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-common-sources + generate-sources + + add-source + + + + ${project.basedir}/../common/src/main/scala + ${project.basedir}/../common/src/main/databricks/scala + + + + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.rat + apache-rat-plugin + + + + diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index f32ead4f3f9..14108a24d9b 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -549,6 +549,31 @@ delta-lake/delta-stub + + + release350db + + + buildver + 350db + + + + + 3.4.4 + spark350db + ${spark350db.version} + ${spark350db.version} + 3.3.1 + true + 1.12.0 + ${spark330.iceberg.version} + + + shim-deps/databricks + delta-lake/delta-spark350db + + release351 @@ -781,6 +806,7 @@ 3.3.0-databricks 3.3.2-databricks 3.4.1-databricks + 3.5.0-databricks 3.5.0 3.5.1 3.5.2 diff --git a/scala2.13/shim-deps/pom.xml b/scala2.13/shim-deps/pom.xml index aba372b6b49..d8a72da7afd 100644 --- a/scala2.13/shim-deps/pom.xml +++ b/scala2.13/shim-deps/pom.xml @@ -159,6 +159,59 @@ + + release350db + + + buildver + 350db + + + + + org.apache.logging.log4j + log4j-core + ${spark.version} + provided + + + org.apache.parquet + parquet-format-internal_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-common-utils_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-sql-api_${scala.binary.version} + ${spark.version} + provided + + + shaded.parquet.org.apache.thrift + shaded-parquet-thrift_${scala.binary.version} + ${spark.version} + provided + + + org.apache.avro + avro-connector + ${spark.version} + provided + + + org.scala-lang.modules + scala-collection-compat_${scala.binary.version} + ${spark.version} + provided + + + dbdeps @@ -194,4 +247,4 @@ - \ No newline at end of file + diff --git a/shim-deps/pom.xml b/shim-deps/pom.xml index 6946cd90168..c16b8eb56f0 100644 --- a/shim-deps/pom.xml +++ b/shim-deps/pom.xml @@ -159,6 +159,59 @@ + + release350db + + + buildver + 350db + + + + + org.apache.logging.log4j + log4j-core + ${spark.version} + provided + + + org.apache.parquet + parquet-format-internal_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-common-utils_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-sql-api_${scala.binary.version} + ${spark.version} + provided + + + shaded.parquet.org.apache.thrift + shaded-parquet-thrift_${scala.binary.version} + ${spark.version} + provided + + + org.apache.avro + avro-connector + ${spark.version} + provided + + + org.scala-lang.modules + scala-collection-compat_${scala.binary.version} + ${spark.version} + provided + + + dbdeps @@ -194,4 +247,4 @@ - \ No newline at end of file + 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 index 780cd6dc5fc..ae8685af621 100644 --- 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 @@ -99,8 +99,8 @@ object GpuPartitioningUtils extends SQLConfHelper { typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, basePaths = basePaths, userSpecifiedSchema = userSpecifiedSchema, - caseSensitive = sparkSession.sqlContext.conf.caseSensitiveAnalysis, - validatePartitionColumns = sparkSession.sqlContext.conf.validatePartitionColumns, + caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis, + validatePartitionColumns = sparkSession.sessionState.conf.validatePartitionColumns, timeZoneId = timeZoneId) (parsed, anyReplacedBase) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala index 5589bca0435..99bbcb199d2 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala @@ -242,7 +242,7 @@ abstract class GpuDataSourceBase( // This is a non-streaming file based datasource. case (format: FileFormat, _) => - val useCatalogFileIndex = sparkSession.sqlContext.conf.manageFilesourcePartitions && + val useCatalogFileIndex = sparkSession.sessionState.conf.manageFilesourcePartitions && catalogTable.isDefined && catalogTable.get.tracksPartitionsInCatalog && catalogTable.get.partitionColumnNames.nonEmpty val (fileCatalog, dataSchema, partitionSchema) = if (useCatalogFileIndex) { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala index 1e38e2ab74e..f9d0b3fc656 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala @@ -136,7 +136,7 @@ case class GpuInMemoryTableScanExec( override def outputOrdering: Seq[SortOrder] = relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder]) - lazy val enableAccumulatorsForTest: Boolean = sparkSession.sqlContext + lazy val enableAccumulatorsForTest: Boolean = sparkSession.sessionState .conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala index 3c3bf8ce3dc..1b12f01f84b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala @@ -38,7 +38,6 @@ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.internal.{config, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle.{ShuffleWriter, _} import org.apache.spark.shuffle.api._ @@ -244,10 +243,8 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( maxBytesInFlight: Long, shuffleExecutorComponents: ShuffleExecutorComponents, numWriterThreads: Int) - extends ShuffleWriter[K, V] - with RapidsShuffleWriterShimHelper - with Logging { - private var myMapStatus: Option[MapStatus] = None + extends RapidsShuffleWriter[K, V] + with RapidsShuffleWriterShimHelper { private val metrics = handle.metrics private val serializationTimeMetric = metrics.get("rapidsShuffleSerializationTime") @@ -265,14 +262,6 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( private val transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true) private val fileBufferSize = sparkConf.get(config.SHUFFLE_FILE_BUFFER_SIZE).toInt * 1024 private val limiter = new BytesInFlightLimiter(maxBytesInFlight) - /** - * Are we in the process of stopping? Because map tasks can call stop() with success = true - * and then call stop() with success = false if they get an exception, we want to make sure - * we don't try deleting files, etc twice. - */ - private var stopping = false - - private val diskBlockObjectWriters = new mutable.HashMap[Int, (Int, DiskBlockObjectWriter)]() /** * Simple wrapper that tracks the time spent iterating the given iterator. @@ -451,7 +440,7 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( shuffleCombineTimeMetric.foreach(_ += combineTimeNs) pl } - myMapStatus = Some(MapStatus(blockManager.shuffleServerId, partLengths, mapId)) + myMapStatus = Some(getMapStatus(blockManager.shuffleServerId, partLengths, mapId)) } catch { // taken directly from BypassMergeSortShuffleWriter case e: Exception => @@ -536,36 +525,7 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( } } - override def stop(success: Boolean): Option[MapStatus] = { - if (stopping) { - None - } else { - stopping = true - if (success) { - if (myMapStatus.isEmpty) { - // should not happen, but adding it just in case (this differs from Spark) - cleanupTempData() - throw new IllegalStateException("Cannot call stop(true) without having called write()"); - } - myMapStatus - } else { - cleanupTempData() - None - } - } - } - private def cleanupTempData(): Unit = { - // The map task failed, so delete our output data. - try { - diskBlockObjectWriters.values.foreach { case (_, writer) => - val file = writer.revertPartialWritesAndClose() - if (!file.delete()) logError(s"Error while deleting file ${file.getAbsolutePath()}") - } - } finally { - diskBlockObjectWriters.clear() - } - } def getBytesInFlight: Long = limiter.getBytesInFlight } @@ -1085,10 +1045,7 @@ class RapidsCachingWriter[K, V]( catalog: ShuffleBufferCatalog, rapidsShuffleServer: Option[RapidsShuffleServer], metrics: Map[String, SQLMetric]) - extends ShuffleWriter[K, V] - with Logging { - private val numParts = handle.dependency.partitioner.numPartitions - private val sizes = new Array[Long](numParts) + extends RapidsCachingWriterBase[K, V](blockManager, handle, mapId, rapidsShuffleServer, catalog) { private val uncompressedMetric: SQLMetric = metrics("dataSize") @@ -1177,41 +1134,6 @@ class RapidsCachingWriter[K, V]( } } - /** - * Used to remove shuffle buffers when the writing task detects an error, calling `stop(false)` - */ - private def cleanStorage(): Unit = { - catalog.removeCachedHandles() - } - - override def stop(success: Boolean): Option[MapStatus] = { - val nvtxRange = new NvtxRange("RapidsCachingWriter.close", NvtxColor.CYAN) - try { - if (!success) { - cleanStorage() - None - } else { - // upon seeing this port, the other side will try to connect to the port - // in order to establish an UCX endpoint (on demand), if the topology has "rapids" in it. - val shuffleServerId = if (rapidsShuffleServer.isDefined) { - val originalShuffleServerId = rapidsShuffleServer.get.originalShuffleServerId - val server = rapidsShuffleServer.get - BlockManagerId( - originalShuffleServerId.executorId, - originalShuffleServerId.host, - originalShuffleServerId.port, - Some(s"${RapidsShuffleTransport.BLOCK_MANAGER_ID_TOPO_PREFIX}=${server.getPort}")) - } else { - blockManager.shuffleServerId - } - logInfo(s"Done caching shuffle success=$success, server_id=$shuffleServerId, " - + s"map_id=$mapId, sizes=${sizes.mkString(",")}") - Some(MapStatus(shuffleServerId, sizes, mapId)) - } - } finally { - nvtxRange.close() - } - } def getPartitionLengths(): Array[Long] = { throw new UnsupportedOperationException("TODO") diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala index 456155f19f5..6af0ee0a0c2 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala @@ -715,7 +715,7 @@ object InternalColumnarRddConverter extends Logging { val b = batch.getOrElse({ // We have to fall back to doing a slow transition. val converters = new GpuExternalRowToColumnConverter(schema) - val conf = new RapidsConf(df.sqlContext.conf) + val conf = new RapidsConf(df.sqlContext.sparkSession.sessionState.conf) val goal = TargetSize(conf.gpuTargetBatchSizeBytes) input.mapPartitions { rowIter => new ExternalRowToColumnarIterator(rowIter, schema, goal, converters) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index 8a88cc4024d..20414ef0ad5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -176,7 +176,7 @@ object TrampolineUtil { } def getSparkConf(spark: SparkSession): SQLConf = { - spark.sqlContext.conf + spark.sessionState.conf } def setExecutorEnv(sc: SparkContext, key: String, value: String): Unit = { diff --git a/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java b/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java index 9692737ab62..c8a993ba55f 100644 --- a/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java +++ b/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/XxHash64Shims.scala b/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/XxHash64Shims.scala index 2b935793de1..e18107ccc24 100644 --- a/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/XxHash64Shims.scala +++ b/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/XxHash64Shims.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala index 42b7817e87f..d3324c78135 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala @@ -46,4 +46,4 @@ final class CudfUnsafeRow( attributes: Array[Attribute], remapping: Array[Int]) extends CudfUnsafeRowBase(attributes, remapping) -object CudfUnsafeRow extends CudfUnsafeRowTrait \ No newline at end of file +object CudfUnsafeRow extends CudfUnsafeRowTrait diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala index 0973561d56d..ae1d34794b9 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala @@ -35,6 +35,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} @@ -240,4 +241,4 @@ trait CudfUnsafeRowTrait { val bitSetWidthInBytes = calculateBitSetWidthInBytes(attributes.length) alignOffset(offset + bitSetWidthInBytes, 8) } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala index ca69e70fb05..a4e0b6a7520 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala index 32ca03974bf..e9dca9497be 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala @@ -33,6 +33,7 @@ {"spark": "341"} {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala index db57c63ba07..1bce787bdb9 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala index fd7eaf3bd36..ab232eaa9d5 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala @@ -32,6 +32,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala index ca0c8a60c01..d3f5cd57c47 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala @@ -32,6 +32,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala index 4fe744f86fa..18c3098a8f1 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala @@ -35,6 +35,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala index 527c8392993..cb66e2de78a 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala @@ -32,6 +32,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala index d1d5d0de056..5f828e9c361 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala @@ -35,6 +35,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala index 91f9e492ec6..1b5cbd6a098 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala @@ -24,6 +24,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala index 1708cf194bb..b2b2cd9d45e 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala @@ -42,4 +42,4 @@ import org.apache.commons.io.output.NullOutputStream object NullOutputStreamShim { def INSTANCE = NullOutputStream.NULL_OUTPUT_STREAM -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala index d503eb77bb3..b308ee10752 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala index d8031d96309..9c5c04e157c 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -33,6 +33,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala index 5f47bc66ce7..691452b80b7 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala @@ -33,6 +33,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala index 1dffdf66c8c..d47a323d129 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala index f0320101f3a..2782a1820b7 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala @@ -35,6 +35,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala index 86be8bc4f2d..123ec3355d1 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala index 5e371cb4ac4..06b06e90714 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala index acab768cd48..7087871a355 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index ab34a37c625..e17c3f21128 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala index 93f061172af..e1d07079322 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala index 8237da81501..26ead2c698a 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala index 600d42cb0c2..705aafe0a8e 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala @@ -35,6 +35,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala index 443daa815d5..af0eabb7a43 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala index f7f164d530c..4cd0bddab5e 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala index 3d68c919695..c0064dfb26e 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala index c6650255edb..8e22bd9bc2d 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala index 318ca3656b2..c39b7a378ee 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala index 94e54bda913..25fbf223dab 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala new file mode 100644 index 00000000000..905b0f9cbb9 --- /dev/null +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala @@ -0,0 +1,156 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"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": "350"} +{"spark": "351"} +{"spark": "352"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids + +import scala.collection.mutable + +import ai.rapids.cudf.{NvtxColor, NvtxRange} +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.shuffle.{RapidsShuffleServer, RapidsShuffleTransport} + +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle.ShuffleWriter +import org.apache.spark.storage._ + + +abstract class RapidsShuffleWriter[K, V]() + extends ShuffleWriter[K, V] + with Logging { + protected var myMapStatus: Option[MapStatus] = None + protected val diskBlockObjectWriters = new mutable.HashMap[Int, (Int, DiskBlockObjectWriter)]() + /** + * Are we in the process of stopping? Because map tasks can call stop() with success = true + * and then call stop() with success = false if they get an exception, we want to make sure + * we don't try deleting files, etc twice. + */ + private var stopping = false + + def getMapStatus( + loc: BlockManagerId, + uncompressedSizes: Array[Long], + mapTaskId: Long): MapStatus = { + MapStatus(loc, uncompressedSizes, mapTaskId) + } + + override def stop(success: Boolean): Option[MapStatus] = { + if (stopping) { + None + } else { + stopping = true + if (success) { + if (myMapStatus.isEmpty) { + // should not happen, but adding it just in case (this differs from Spark) + cleanupTempData() + throw new IllegalStateException("Cannot call stop(true) without having called write()"); + } + myMapStatus + } else { + cleanupTempData() + None + } + } + } + + private def cleanupTempData(): Unit = { + // The map task failed, so delete our output data. + try { + diskBlockObjectWriters.values.foreach { case (_, writer) => + val file = writer.revertPartialWritesAndClose() + if (!file.delete()) logError(s"Error while deleting file ${file.getAbsolutePath()}") + } + } finally { + diskBlockObjectWriters.clear() + } + } +} + +abstract class RapidsCachingWriterBase[K, V]( + blockManager: BlockManager, + handle: GpuShuffleHandle[K, V], + mapId: Long, + rapidsShuffleServer: Option[RapidsShuffleServer], + catalog: ShuffleBufferCatalog) + extends ShuffleWriter[K, V] + with Logging { + protected val numParts = handle.dependency.partitioner.numPartitions + protected val sizes = new Array[Long](numParts) + + /** + * Used to remove shuffle buffers when the writing task detects an error, calling `stop(false)` + */ + private def cleanStorage(): Unit = { + catalog.removeCachedHandles() + } + + override def stop(success: Boolean): Option[MapStatus] = { + val nvtxRange = new NvtxRange("RapidsCachingWriter.close", NvtxColor.CYAN) + try { + if (!success) { + cleanStorage() + None + } else { + // upon seeing this port, the other side will try to connect to the port + // in order to establish an UCX endpoint (on demand), if the topology has "rapids" in it. + val shuffleServerId = if (rapidsShuffleServer.isDefined) { + val originalShuffleServerId = rapidsShuffleServer.get.originalShuffleServerId + val server = rapidsShuffleServer.get + BlockManagerId( + originalShuffleServerId.executorId, + originalShuffleServerId.host, + originalShuffleServerId.port, + Some(s"${RapidsShuffleTransport.BLOCK_MANAGER_ID_TOPO_PREFIX}=${server.getPort}")) + } else { + blockManager.shuffleServerId + } + logInfo(s"Done caching shuffle success=$success, server_id=$shuffleServerId, " + + s"map_id=$mapId, sizes=${sizes.mkString(",")}") + Some(MapStatus(shuffleServerId, sizes, mapId)) + } + } finally { + nvtxRange.close() + } + } + +} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala index 6ec33305aaa..61373d5c623 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala index c71288f576f..793d6f2b8bd 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index c365ebe5c5d..1123f1013f9 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -105,7 +105,7 @@ case class GpuCreateDataSourceTableAsSelectCommand( result match { case _: HadoopFsRelation if table.partitionColumnNames.nonEmpty && - sparkSession.sqlContext.conf.manageFilesourcePartitions => + sparkSession.sessionState.conf.manageFilesourcePartitions => // Need to recover partitions into the metastore so our saved data is visible. sessionState.executePlan( SparkShimImpl.v1RepairTableCommand(table.identifier)).toRdd diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala index 9482b8f594f..77927eed0d8 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala @@ -35,6 +35,7 @@ {"spark": "341db"} {"spark": "342"} {"spark": "343"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index a7c9d8e2998..c1b4f77b4ac 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala index 4851713ae50..1ab8eead887 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala index f05d318c671..fc62158cb29 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala index f4683903d8a..508136a765d 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala index 0e87f02eb56..412861e236a 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala index b1e787fe64f..31ef6a97350 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala @@ -36,6 +36,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala index c747ae4da55..35e7e5e5461 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala @@ -35,6 +35,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala b/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala index f9b01e0154a..4025ecc7ef7 100644 --- a/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala +++ b/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala index e59573bdcc0..5b072d7beac 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala index 350a1c32109..5d0b9ef74ee 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala index 3a735686c5e..3eecc63cc0e 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala index 4aa9f1c06a9..4785b007336 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala index 109a638d158..f68ca7ba02b 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala index de58de2fa1b..6f6e9e3c04c 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala index 4356f67c576..e502cc5063b 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala index 23cfc01275b..3661c870f11 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala index c5f5803c6c1..efa20be8208 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala index 1278c26d3de..b2e336bafa7 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala index 20bcd108bcd..76a473abd9a 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala @@ -28,6 +28,7 @@ {"spark": "341db"} {"spark": "342"} {"spark": "343"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala index 4be9f914b41..2c7fc6c0256 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala @@ -29,6 +29,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala index 4e8e62e943e..a077a204c85 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala index afb39e22212..f3d2f979206 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala @@ -29,6 +29,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala index 056b44803d1..8cbf45200b8 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala index e8b24305d13..38205039398 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala @@ -27,6 +27,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala index 3fc1ec35166..db4cceb0ba6 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -29,6 +29,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index acbd921232d..5391c474771 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala index 9833a076af2..ffd9bed984e 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index 133a35fe592..0f93aa24074 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 088fd2e8d05..a705775c33f 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala index efd2b47cbfa..03c42269c7d 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala index 52109644655..dffdcdca700 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala new file mode 100644 index 00000000000..62dafd140d5 --- /dev/null +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "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": "350"} +{"spark": "351"} +{"spark": "352"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.datasources.parquet + +import java.time.ZoneId + +import org.apache.parquet.VersionParser.ParsedVersion +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.PageReadStore + +object RapidsVectorizedColumnReader { + def apply(descriptor: ColumnDescriptor, + isRequired: Boolean, + pageReadStore: PageReadStore, + convertTz: ZoneId, + datetimeRebaseMode: String, + datetimeRebaseTz: String, + int96RebaseMode: String, + int96RebaseTz: String, + writerVersion: ParsedVersion) = { + new VectorizedColumnReader( + descriptor, + isRequired, + pageReadStore, + null, + datetimeRebaseMode, + datetimeRebaseTz, + int96RebaseMode, + null, + writerVersion) + } +} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala index 4358badaf76..bbd37ab3675 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} @@ -153,7 +154,7 @@ class ShimCurrentBatchIterator( if (!missingColumns.contains(cv.getColumn)) { if (cv.getColumn.isPrimitive) { val column = cv.getColumn - val reader = new VectorizedColumnReader( + val reader = RapidsVectorizedColumnReader( column.descriptor.get, column.required, pages, diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala index 80c393042f5..8efff9d22c3 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala index 3596dce03b7..a02502992b6 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala index eb20cc42237..129f28c9d12 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala index 980ce817ee4..65b415773f8 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} 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 index 8aa197ff23d..2ab87c47cd1 100644 --- 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 @@ -30,6 +30,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index a601b3e5b7b..be447a7c3a5 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala index 835a0d30dae..2045e3d4f08 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala index e50f922c893..7ce37c2ac6f 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala index 8d5b57aaa55..1b168bf07ee 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala index 38b2138cd33..304cc9a8e11 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala index f4d5ac6a114..4a657cd3270 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala index f286718a893..2eb80c092ce 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala index 33f86513519..6cb7e6fbcd1 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims 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 25eaf71bede..f359cb8fd73 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 @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala index f02d8755a6a..8088285fda1 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala index 25b8a94ec66..5a1d269bed1 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala @@ -17,6 +17,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala index 623484adbc5..0f1027bd53f 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala index f3f8d3630b6..12339ab902c 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala index 8a197620828..5e843fd0894 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala index 38c8db30b81..58516fe741e 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala index bf1a9e2f529..090a6e24e88 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala index b52b64c0e31..9c4a6beb75f 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala index cbc99cd58b7..458e63f1276 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala index 220bec0b763..02d5ae9ae16 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala index fb60f15db4d..e2396feef9c 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala index 0c5594b8da0..0dda467bca9 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala index 44ccc46bdd4..a3c2d1d2342 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/shimExpressions.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/shimExpressions.scala index 4f2b0e80395..88557ed4410 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/shimExpressions.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/shimExpressions.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala index c76a5085dbb..09b2af6d9fd 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala @@ -21,6 +21,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.shims diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala index ccb236e0e6c..cc6fe40d3ef 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala @@ -17,6 +17,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.rapids.shims @@ -55,4 +56,5 @@ abstract class GpuDatabricksShuffleExchangeExecBase( } override def shuffleId: Int = shuffleDependencyColumnar.shuffleId + } diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala index dd04d7cfd26..6761b87fc0d 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala @@ -22,6 +22,7 @@ {"spark": "341db"} {"spark": "342"} {"spark": "343"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.parquet @@ -44,7 +45,7 @@ object ParquetCVShims { val defaultValue = if (sparkSchema != null) { getExistenceDefaultValues(sparkSchema) } else null - new ParquetColumnVector(column, vector, capacity, memoryMode, missingColumns, isTopLevel, + ShimParquetColumnVector(column, vector, capacity, memoryMode, missingColumns, isTopLevel, defaultValue) } } diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala new file mode 100644 index 00000000000..d71d6d3a164 --- /dev/null +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "330db"} +{"spark": "332db"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.memory.MemoryMode +import org.apache.spark.sql.execution.vectorized.WritableColumnVector + +object ShimParquetColumnVector { + def apply( + column: ParquetColumn, + vector: WritableColumnVector, + capacity: Int, + memoryMode: MemoryMode, + missingColumns: java.util.Set[ParquetColumn], + isTopLevel: Boolean, + defaultValue: Any): ParquetColumnVector = { + new ParquetColumnVector(column, vector, capacity, memoryMode, missingColumns, isTopLevel, + defaultValue) + } +} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala index e75edc7a772..1fa74e079fc 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala index ca3fbdb7feb..3d0eeb8eb25 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala @@ -28,6 +28,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala index e8b6736a650..c69a012517a 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala index 9b8b7e4693c..a8dd471a33c 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index 08bab08e4dc..5703c860504 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index 9c9784fc31c..a00e6641a25 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala index 5fabf05069f..ef10f2e2a93 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala index b090b2b8189..6e11c43ce5d 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala index 00894db1e6f..ac2c910a5ea 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala index 53c22f3d53d..856f0032510 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala @@ -18,6 +18,7 @@ {"spark": "330db"} {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index 7309529b081..2aad5a89435 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} 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 index 6d5fdff07d8..525b431d08d 100644 --- 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 @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala index 5b7bf425920..43390731748 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index 1b9efd417d0..c5f0a7be0ee 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala index fb017647024..bb4870dd4ec 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala index 393b168d490..b1724525c8d 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala index 43ef6118746..713eaba139b 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "332db"} {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala index fc8094b793f..856b424c04d 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala index fdfaaffeb56..eec6321b1dc 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala @@ -18,6 +18,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala index ca463559319..1d66231ff6e 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala @@ -23,6 +23,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala index c3996a83b78..24470ba51b1 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 29c72188635..8d161c81764 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index b44be532b52..1049def0064 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} @@ -98,7 +99,7 @@ case class GpuCreateDataSourceTableAsSelectCommand( result match { case _: HadoopFsRelation if table.partitionColumnNames.nonEmpty && - sparkSession.sqlContext.conf.manageFilesourcePartitions => + sparkSession.sessionState.conf.manageFilesourcePartitions => // Need to recover partitions into the metastore so our saved data is visible. sessionState.executePlan( SparkShimImpl.v1RepairTableCommand(table.identifier)).toRdd @@ -135,4 +136,4 @@ case class GpuCreateDataSourceTableAsSelectCommand( throw ex } } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala index 37395a72d2f..8a2721b84ec 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala index bff6af379b7..f97346e6c14 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala @@ -21,6 +21,7 @@ {"spark": "341db"} {"spark": "342"} {"spark": "343"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala index d70e5c3c08b..3f696f75744 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala index 7b2fd52263b..93a05a6838c 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala index 59ac9bd84f5..bace9424c2d 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala index f789f2d31cb..78d42ecae9f 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala index 408d18ece70..6922315eaeb 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala index 18da13783a3..b862c6724b7 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala @@ -19,6 +19,7 @@ {"spark": "341db"} {"spark": "342"} {"spark": "343"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala index 57e77499064..5c449176daa 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala @@ -22,6 +22,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala index beee3bb20f6..ae83cd1b732 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala index d927709160c..1e2835cf286 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala index 6856b52cba9..f19127b3dc6 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala index 2ccf3d8eb8a..767fd0ab625 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index ddf74821382..54f10835bd5 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala index d8f0cc2747b..71b05c8f87a 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala index 47b8158d0cb..a8d3b8a115a 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala index 7961fcc35c0..2bfc5116c95 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala index fd7c9b505ce..5ae550a50ca 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala @@ -20,6 +20,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala index 1bfe67cc3f2..5355a50ec2d 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala index 4c89b323453..75729326c8a 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala index 5b3bfe2137b..70b09e75941 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index bf5567298bf..5257f93a633 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala index 4e5e01fabaf..13fb8f3bce1 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala index 667a6912abc..8b1fc7f48b8 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala index b70540cc271..2a3a53c3fcb 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala index b2ebcd50617..8bf4dae4e76 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala index 398bc8e187d..0c63005443b 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.v2.rapids diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala index d1380facb86..d88d1a6fe86 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.v2.rapids diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala index b52d9e4695c..966a92ec657 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala index cbdf18ade4c..3777e36aa67 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala @@ -18,6 +18,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala index 72cf48427db..44368c4d9c5 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala index 72a640a5330..50c328749a5 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala @@ -18,6 +18,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala index fd5186fecc8..c9f345e98cd 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala @@ -18,6 +18,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/ShuffleExchangeShim.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/ShuffleExchangeShim.scala index 5c6b5b21746..37b4f9005d1 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/ShuffleExchangeShim.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/ShuffleExchangeShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala index a7c4d320fc4..2b6a3ebe224 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala index 50c5e280e9c..5f41de0e2a9 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala index 940a48bc225..431f2722b02 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala index 0317a89009e..6a3ad33ac32 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala index 42c6178ff83..00df80f19a6 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala @@ -19,6 +19,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala index 63a4289c5b0..6e8452e3403 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 9b800d4e51a..a1490eb0959 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala b/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala index 23759510d08..cfad91e34ee 100644 --- a/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala +++ b/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "342"} {"spark": "343"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala index 429e04162de..53a8c629ba1 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala index 3d41dd868ff..28ddc9264dc 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} @@ -31,4 +32,4 @@ object LegacyBehaviorPolicyShim { def isLegacyTimeParserPolicy(): Boolean = { SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala index caf337d5458..748fb5694fc 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} @@ -26,4 +27,4 @@ import org.apache.commons.io.output.NullOutputStream object NullOutputStreamShim { def INSTANCE = NullOutputStream.INSTANCE -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala index 47d3969e6cb..966a27e3f12 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala @@ -61,4 +61,4 @@ object PythonMapInArrowExecShims { }) ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala index 4d5a0552961..9981f60c374 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} @@ -37,4 +38,4 @@ object ArrowUtilsShim { errorOnDuplicatedFieldNames: Boolean = true, largeVarTypes: Boolean = false): Schema = { ArrowUtils.toArrowSchema(schema, timeZoneId, errorOnDuplicatedFieldNames, largeVarTypes) } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala index b07ccb43c79..84881cd4c40 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} @@ -32,4 +33,4 @@ object DataTypeUtilsShim { def toAttributes(structType: StructType): Seq[Attribute] = DataTypeUtils.toAttributes(structType) -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala similarity index 93% rename from sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala rename to sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala index e6c26eb65b8..aa27ea9fbb4 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala @@ -15,14 +15,13 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec class BatchScanExecMeta(p: BatchScanExec, diff --git a/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala new file mode 100644 index 00000000000..c7ba0aea001 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2020-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": "350db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.unsafe.types.VariantVal +import org.apache.spark.unsafe.types.geo._ + + +final class CudfUnsafeRow( + attributes: Array[Attribute], + remapping: Array[Int]) extends CudfUnsafeRowBase(attributes, remapping) { + + def getGeography(x$1: Int): GeographyVal = { + throw new UnsupportedOperationException("Not Implemented yet") + } + + def getGeometry(x$1: Int): GeometryVal = { + throw new UnsupportedOperationException("Not Implemented yet") + } + + def getVariant(x$1: Int): VariantVal = { + throw new UnsupportedOperationException("Not Implemented yet") + } + +} + +object CudfUnsafeRow extends CudfUnsafeRowTrait diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala similarity index 89% rename from sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala rename to sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index c2d2c8d934e..ca6b87fe6c4 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -25,15 +26,14 @@ import com.nvidia.spark.rapids.GpuScan import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, DynamicPruningExpression, Expression, Literal, RowOrdering, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, DynamicPruningExpression, Expression, Literal, SortOrder} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, KeyGroupedShuffleSpec, Partitioning, SinglePartition} +import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, Partitioning, SinglePartition} import org.apache.spark.sql.catalyst.util.{truncatedString, InternalRowComparableWrapper} import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.execution.datasources.rapids.DataSourceStrategyUtils import org.apache.spark.sql.execution.datasources.v2.{DataSourceRDD, StoragePartitionJoinParams} -import org.apache.spark.sql.internal.SQLConf case class GpuBatchScanExec( output: Seq[AttributeReference], @@ -166,18 +166,6 @@ case class GpuBatchScanExec( (groupedParts, expressions) } - // Also re-group the partitions if we are reducing compatible partition expressions - val finalGroupedPartitions = spjParams.reducers match { - case Some(reducers) => - val result = groupedPartitions.groupBy { case (row, _) => - KeyGroupedShuffleSpec.reducePartitionValue(row, partExpressions, reducers) - }.map { case (wrapper, splits) => (wrapper.row, splits.flatMap(_._2)) }.toSeq - val rowOrdering = RowOrdering.createNaturalAscendingOrdering( - partExpressions.map(_.dataType)) - result.sorted(rowOrdering.on((t: (InternalRow, _)) => t._1)) - case _ => groupedPartitions - } - // When partially clustered, the input partitions are not grouped by partition // values. Here we'll need to check `commonPartitionValues` and decide how to group // and replicate splits within a partition. @@ -185,15 +173,15 @@ case class GpuBatchScanExec( // A mapping from the common partition values to how many splits the partition // should contain. val commonPartValuesMap = spjParams.commonPartitionValues - .get - .map(t => (InternalRowComparableWrapper(t._1, partExpressions), t._2)) - .toMap - val nestGroupedPartitions = finalGroupedPartitions.map { case (partValue, splits) => + .get + .map(t => (InternalRowComparableWrapper(t._1, partExpressions), t._2)) + .toMap + val nestGroupedPartitions = groupedPartitions.map { case (partValue, splits) => // `commonPartValuesMap` should contain the part value since it's the super set. val numSplits = commonPartValuesMap - .get(InternalRowComparableWrapper(partValue, partExpressions)) + .get(InternalRowComparableWrapper(partValue, partExpressions)) assert(numSplits.isDefined, s"Partition value $partValue does not exist in " + - "common partition values from Spark plan") + "common partition values from Spark plan") val newSplits = if (spjParams.replicatePartitions) { // We need to also replicate partitions according to the other side of join @@ -221,7 +209,7 @@ case class GpuBatchScanExec( } else { // either `commonPartitionValues` is not defined, or it is defined but // `applyPartialClustering` is false. - val partitionMapping = finalGroupedPartitions.map { case (partValue, splits) => + val partitionMapping = groupedPartitions.map { case (partValue, splits) => InternalRowComparableWrapper(partValue, partExpressions) -> splits }.toMap diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala similarity index 98% rename from sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala rename to sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala index de8e98962a7..9d23ef5665f 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala @@ -16,6 +16,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala new file mode 100644 index 00000000000..370c39f1279 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.paths.SparkPath +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile + +trait PartitionedFileUtilsShimBase { + + // Wrapper for case class constructor so Java code can access + // the default values across Spark versions. + def newPartitionedFile(partitionValues: InternalRow, + filePath: String, + start: Long, + length: Long): PartitionedFile = PartitionedFile(partitionValues, + SparkPath.fromPathString(filePath), start, length) + + def withNewLocations(pf: PartitionedFile, locations: Seq[String]): PartitionedFile = { + pf.copy(locations = locations) + } +} diff --git a/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala new file mode 100644 index 00000000000..ef348028c94 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2023-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": "350db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.python.MapInArrowExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.GpuPythonMapInArrowExecMeta +import org.apache.spark.sql.types.{BinaryType, StringType} + +object PythonMapInArrowExecShims { + + def execs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( + GpuOverrides.exec[MapInArrowExec]( + "The backend for Map Arrow Iterator UDF. Accelerates the data transfer between the" + + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all), + (mapPy, conf, p, r) => new GpuPythonMapInArrowExecMeta(mapPy, conf, p, r) { + override def tagPlanForGpu(): Unit = { + super.tagPlanForGpu() + if (SQLConf.get.getConf(SQLConf.ARROW_EXECUTION_USE_LARGE_VAR_TYPES)) { + + val inputTypes = mapPy.child.schema.fields.map(_.dataType) + val outputTypes = mapPy.output.map(_.dataType) + + val hasStringOrBinaryTypes = (inputTypes ++ outputTypes).exists(dataType => + TrampolineUtil.dataTypeExistsRecursively(dataType, + dt => dt == StringType || dt == BinaryType)) + + if (hasStringOrBinaryTypes) { + willNotWorkOnGpu(s"${SQLConf.ARROW_EXECUTION_USE_LARGE_VAR_TYPES.key} is " + + s"enabled and the schema contains string or binary types. This is not " + + s"supported on the GPU.") + } + } + } + }) + ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap + +} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala similarity index 79% rename from sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala rename to sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala index 70d40fc19a0..c3e0b747c92 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala @@ -14,14 +14,14 @@ * limitations under the License. */ /*** spark-rapids-shim-json-lines +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{ExprRule, GpuOverrides} -import com.nvidia.spark.rapids.{ExprChecks, GpuExpression, TypeSig, UnaryExprMeta} +import com.nvidia.spark.rapids.ExprRule -import org.apache.spark.sql.catalyst.expressions.{Expression, RaiseError} +import org.apache.spark.sql.catalyst.expressions.Expression object RaiseErrorShim { val exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Map.empty diff --git a/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/spark350db/SparkShimServiceProvider.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/spark350db/SparkShimServiceProvider.scala new file mode 100644 index 00000000000..8beb616ed4c --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/shims/spark350db/SparkShimServiceProvider.scala @@ -0,0 +1,37 @@ +/* + * Copyright (c) 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims.spark350db + +import com.nvidia.spark.rapids.{DatabricksShimVersion, ShimVersion} + +import org.apache.spark.SparkEnv + +object SparkShimServiceProvider { + val VERSION = DatabricksShimVersion(3, 5, 0) +} + +class SparkShimServiceProvider extends com.nvidia.spark.rapids.SparkShimServiceProvider { + + override def getShimVersion: ShimVersion = SparkShimServiceProvider.VERSION + + def matchesVersion(version: String): Boolean = { + SparkEnv.get.conf.get("spark.databricks.clusterUsageTags.sparkVersion", "").startsWith("14.3.") + } +} diff --git a/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/spark350db/RapidsShuffleManager.scala b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/spark350db/RapidsShuffleManager.scala new file mode 100644 index 00000000000..b3c05f26ae3 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/com/nvidia/spark/rapids/spark350db/RapidsShuffleManager.scala @@ -0,0 +1,28 @@ +/* + * Copyright (c) 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.spark350db + +import org.apache.spark.SparkConf +import org.apache.spark.sql.rapids.ProxyRapidsShuffleInternalManagerBase + +/** A shuffle manager optimized for the RAPIDS Plugin for Apache Spark. */ +sealed class RapidsShuffleManager( + conf: SparkConf, + isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/spark350db/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala new file mode 100644 index 00000000000..892b21a0c79 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.rapids.shims + +import com.nvidia.spark.rapids.GpuPartitioning + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveRepartitioningStatus +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin} +import org.apache.spark.sql.rapids.execution.ShuffledBatchRDD + +case class GpuShuffleExchangeExec( + gpuOutputPartitioning: GpuPartitioning, + child: SparkPlan, + shuffleOrigin: ShuffleOrigin)( + cpuOutputPartitioning: Partitioning) + extends GpuDatabricksShuffleExchangeExecBase(gpuOutputPartitioning, child, shuffleOrigin)( + cpuOutputPartitioning) { + + override def getShuffleRDD( + partitionSpecs: Array[ShufflePartitionSpec], + lazyFetching: Boolean): RDD[_] = { + new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs) + } + + // DB SPECIFIC - throw if called since we don't know how its used + override def withNewNumPartitions(numPartitions: Int): ShuffleExchangeLike = { + throw new UnsupportedOperationException + } + + // DB SPECIFIC - throw if called since we don't know how its used + override def targetOutputPartitioning: Partitioning = { + throw new UnsupportedOperationException + } + + def adaptiveRepartitioningStatus(): AdaptiveRepartitioningStatus = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + def repartition(numPartitions: Int,updatedRepartitioningStatus: AdaptiveRepartitioningStatus): + ShuffleExchangeLike = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } +} diff --git a/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala new file mode 100644 index 00000000000..42027c9b0d3 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala @@ -0,0 +1,51 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.datasources.parquet + +import java.time.ZoneId + +import org.apache.parquet.VersionParser.ParsedVersion +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.PageReadStore + +object RapidsVectorizedColumnReader { + def apply(descriptor: ColumnDescriptor, + isRequired: Boolean, + pageReadStore: PageReadStore, + convertTz: ZoneId, + datetimeRebaseMode: String, + datetimeRebaseTz: String, + int96RebaseMode: String, + int96RebaseTz: String, + writerVersion: ParsedVersion) = { + val useNativeDictionary = false + new VectorizedColumnReader( + descriptor, + useNativeDictionary, + isRequired, + pageReadStore, + null, + datetimeRebaseMode, + datetimeRebaseTz, + int96RebaseMode, + null, + writerVersion) + } +} diff --git a/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala new file mode 100644 index 00000000000..eaa246c56b9 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.memory.MemoryMode +import org.apache.spark.sql.execution.vectorized.WritableColumnVector + +object ShimParquetColumnVector { + def apply( + column: ParquetColumn, + vector: WritableColumnVector, + capacity: Int, + memoryMode: MemoryMode, + missingColumns: java.util.Set[ParquetColumn], + isTopLevel: Boolean, + defaultValue: Any): ParquetColumnVector = { + new ParquetColumnVector(column, vector, capacity, memoryMode, missingColumns, isTopLevel, + defaultValue, "") + } +} diff --git a/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala new file mode 100644 index 00000000000..0b6bba0dba9 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala @@ -0,0 +1,75 @@ +/* + * 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.datasources.parquet.rapids.shims + +import java.time.ZoneId +import java.util.TimeZone + +import org.apache.parquet.VersionParser.ParsedVersion +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{GroupType, Type} + +import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec +import org.apache.spark.sql.execution.datasources.parquet.{ParentContainerUpdater, ParquetRowConverter, ParquetToSparkSchemaConverter, VectorizedColumnReader} +import org.apache.spark.sql.internal.LegacyBehaviorPolicy +import org.apache.spark.sql.types.StructType + +class ShimParquetRowConverter( + schemaConverter: ParquetToSparkSchemaConverter, + parquetType: GroupType, + catalystType: StructType, + convertTz: Option[ZoneId], + datetimeRebaseMode: String, // always LegacyBehaviorPolicy.CORRECTED + int96RebaseMode: String, // always LegacyBehaviorPolicy.EXCEPTION + int96CDPHive3Compatibility: Boolean, + updater: ParentContainerUpdater +) extends ParquetRowConverter( + schemaConverter, + parquetType, + catalystType, + convertTz, + // no need to rebase, so set originTimeZone as default + RebaseSpec(LegacyBehaviorPolicy.withName(datetimeRebaseMode)), + // no need to rebase, so set originTimeZone as default + RebaseSpec(LegacyBehaviorPolicy.withName(int96RebaseMode)), + updater) + +class ShimVectorizedColumnReader( + index: Int, + columns: java.util.List[ColumnDescriptor], + types: java.util.List[Type], + pageReadStore: PageReadStore, + convertTz: ZoneId, + datetimeRebaseMode: String, // always LegacyBehaviorPolicy.CORRECTED + int96RebaseMode: String, // always LegacyBehaviorPolicy.EXCEPTION + int96CDPHive3Compatibility: Boolean, + writerVersion: ParsedVersion +) extends VectorizedColumnReader( + columns.get(index), + true, + false, + pageReadStore, + convertTz, + datetimeRebaseMode, + TimeZone.getDefault.getID, // use default zone because of no rebase + int96RebaseMode, + TimeZone.getDefault.getID, // use default zone because of will throw exception if rebase + writerVersion) diff --git a/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala new file mode 100644 index 00000000000..976c68af66f --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala @@ -0,0 +1,132 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids + +import scala.collection.mutable + +import ai.rapids.cudf.{NvtxColor, NvtxRange} +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.shuffle.{RapidsShuffleServer, RapidsShuffleTransport} + +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.MapStatusWithStats +import org.apache.spark.shuffle.ShuffleWriter +import org.apache.spark.storage._ + +abstract class RapidsShuffleWriter[K, V]() + extends ShuffleWriter[K, V] + with Logging { + protected var myMapStatus: Option[MapStatusWithStats] = None + protected val diskBlockObjectWriters = new mutable.HashMap[Int, (Int, DiskBlockObjectWriter)]() + /** + * Are we in the process of stopping? Because map tasks can call stop() with success = true + * and then call stop() with success = false if they get an exception, we want to make sure + * we don't try deleting files, etc twice. + */ + private var stopping = false + + def getMapStatus( + loc: BlockManagerId, + uncompressedSizes: Array[Long], + mapTaskId: Long): MapStatusWithStats = { + MapStatusWithStats(loc, uncompressedSizes, mapTaskId) + } + + override def stop(success: Boolean): Option[MapStatusWithStats] = { + if (stopping) { + None + } else { + stopping = true + if (success) { + if (myMapStatus.isEmpty) { + // should not happen, but adding it just in case (this differs from Spark) + cleanupTempData() + throw new IllegalStateException("Cannot call stop(true) without having called write()"); + } + myMapStatus + } else { + cleanupTempData() + None + } + } + } + + private def cleanupTempData(): Unit = { + // The map task failed, so delete our output data. + try { + diskBlockObjectWriters.values.foreach { case (_, writer) => + val file = writer.revertPartialWritesAndClose() + if (!file.delete()) logError(s"Error while deleting file ${file.getAbsolutePath()}") + } + } finally { + diskBlockObjectWriters.clear() + } + } +} + +abstract class RapidsCachingWriterBase[K, V]( + blockManager: BlockManager, + handle: GpuShuffleHandle[K, V], + mapId: Long, + rapidsShuffleServer: Option[RapidsShuffleServer], + catalog: ShuffleBufferCatalog) + extends ShuffleWriter[K, V] + with Logging { + protected val numParts = handle.dependency.partitioner.numPartitions + protected val sizes = new Array[Long](numParts) + + /** + * Used to remove shuffle buffers when the writing task detects an error, calling `stop(false)` + */ + private def cleanStorage(): Unit = { + catalog.removeCachedHandles() + } + + override def stop(success: Boolean): Option[MapStatusWithStats] = { + val nvtxRange = new NvtxRange("RapidsCachingWriter.close", NvtxColor.CYAN) + try { + if (!success) { + cleanStorage() + None + } else { + // upon seeing this port, the other side will try to connect to the port + // in order to establish an UCX endpoint (on demand), if the topology has "rapids" in it. + val shuffleServerId = if (rapidsShuffleServer.isDefined) { + val originalShuffleServerId = rapidsShuffleServer.get.originalShuffleServerId + val server = rapidsShuffleServer.get + BlockManagerId( + originalShuffleServerId.executorId, + originalShuffleServerId.host, + originalShuffleServerId.port, + Some(s"${RapidsShuffleTransport.BLOCK_MANAGER_ID_TOPO_PREFIX}=${server.getPort}")) + } else { + blockManager.shuffleServerId + } + logInfo(s"Done caching shuffle success=$success, server_id=$shuffleServerId, " + + s"map_id=$mapId, sizes=${sizes.mkString(",")}") + Some(MapStatusWithStats(shuffleServerId, sizes, mapId)) + } + } finally { + nvtxRange.close() + } + } + +} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala similarity index 98% rename from sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala rename to sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala index c16564f523e..43690bf61e5 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -14,6 +14,7 @@ * limitations under the License. */ /*** spark-rapids-shim-json-lines +{"spark": "350db"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala new file mode 100644 index 00000000000..015beea5844 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala @@ -0,0 +1,72 @@ +/* + * 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": "350db"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.api.python.PythonEvalType +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, PythonUDF} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.python.MapInArrowExec +import org.apache.spark.sql.rapids.execution.python.GpuMapInBatchExec + +class GpuPythonMapInArrowExecMetaBase( + mapArrow: MapInArrowExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends SparkPlanMeta[MapInArrowExec](mapArrow, conf, parent, rule) { + + override def replaceMessage: String = "partially run on GPU" + override def noReplacementPossibleMessage(reasons: String): String = + s"cannot run even partially on the GPU because $reasons" + + protected val udf: BaseExprMeta[PythonUDF] = GpuOverrides.wrapExpr( + mapArrow.func.asInstanceOf[PythonUDF], conf, Some(this)) + protected val resultAttrs: Seq[BaseExprMeta[Attribute]] = + mapArrow.output.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + + override val childExprs: Seq[BaseExprMeta[_]] = resultAttrs :+ udf + + override def convertToGpu(): GpuExec = + GpuPythonMapInArrowExec( + udf.convertToGpu(), + resultAttrs.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], + childPlans.head.convertIfNeeded(), + isBarrier = false, + ) +} + +/* + * A relation produced by applying a function that takes an iterator of PyArrow's record + * batches and outputs an iterator of PyArrow's record batches. + * + * This GpuMapInPandasExec aims at accelerating the data transfer between + * JVM and Python, and scheduling GPU resources for its Python processes. + * + */ +case class GpuPythonMapInArrowExec( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + override val isBarrier: Boolean) extends GpuMapInBatchExec { + + override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_ARROW_ITER_UDF +} diff --git a/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala new file mode 100644 index 00000000000..57b85e6b466 --- /dev/null +++ b/sql-plugin/src/main/spark350db/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.execution.python._ + +class GpuPythonMapInArrowExecMeta( + mapArrow: MapInArrowExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends GpuPythonMapInArrowExecMetaBase(mapArrow, conf, parent, rule) { + +} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala index 623005654fc..77f28bd77d5 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala @@ -30,4 +30,4 @@ final class CudfUnsafeRow( } } -object CudfUnsafeRow extends CudfUnsafeRowTrait \ No newline at end of file +object CudfUnsafeRow extends CudfUnsafeRowTrait diff --git a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/ConcurrentWriterMetricsSuite.scala b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/ConcurrentWriterMetricsSuite.scala index cad9da2d33a..29faf9b1233 100644 --- a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/ConcurrentWriterMetricsSuite.scala +++ b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/ConcurrentWriterMetricsSuite.scala @@ -35,6 +35,7 @@ {"spark": "341db"} {"spark": "342"} {"spark": "343"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala index 31e269affa4..e93b0c41bc9 100644 --- a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala +++ b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala @@ -33,6 +33,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/tests/src/test/spark330/scala/com/nvidia/spark/rapids/BloomFilterAggregateQuerySuite.scala b/tests/src/test/spark330/scala/com/nvidia/spark/rapids/BloomFilterAggregateQuerySuite.scala index 7fc26e76005..680b4147e06 100644 --- a/tests/src/test/spark330/scala/com/nvidia/spark/rapids/BloomFilterAggregateQuerySuite.scala +++ b/tests/src/test/spark330/scala/com/nvidia/spark/rapids/BloomFilterAggregateQuerySuite.scala @@ -29,6 +29,7 @@ {"spark": "341db"} {"spark": "342"} {"spark": "343"} +{"spark": "350db"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala index b508af3021c..aa5d594d4b3 100644 --- a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala +++ b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala @@ -21,6 +21,7 @@ {"spark": "342"} {"spark": "343"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"} diff --git a/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala b/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala index d6fa7dfdd27..653f26c207f 100644 --- a/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala +++ b/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350"} +{"spark": "350db"} {"spark": "351"} {"spark": "352"} {"spark": "400"}