From 906e5cd930fc7384dc1edeec59ded242df654e35 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 12 May 2023 08:01:57 -0600 Subject: [PATCH 01/68] prep for 332db Signed-off-by: Andy Grove --- jenkins/databricks/build.sh | 52 +++++++++++++++++-- pom.xml | 42 ++++++++++++++- ...aSourceTableAsSelectCommandMetaShims.scala | 1 + .../rapids/shims/FileIndexOptionsShims.scala | 1 + .../spark/rapids/shims/GlobalLimitShims.scala | 1 + ...dCreateHiveTableAsSelectCommandShims.scala | 1 + .../rapids/shims/OrcProtoWriterShim.scala | 1 + .../shims/ParquetLegacyNanoAsLongShims.scala | 1 + .../shims/ParquetTimestampNTZShims.scala | 1 + .../rapids/shims/GpuInsertIntoHiveTable.scala | 1 + .../rapids/shims/HiveProviderCmdShims.scala | 1 + .../spark/sql/rapids/GpuDataSource.scala | 1 + .../sql/rapids/GpuFileFormatWriter.scala | 1 + ...eDataSourceTableAsSelectCommandShims.scala | 1 + .../sql/rapids/shims/SchemaUtilsShims.scala | 1 + .../shims/ShimSupportsRuntimeFiltering.java | 1 + .../spark/rapids/shims/GpuParquetCrypto.scala | 1 + .../nvidia/spark/rapids/shims/HashUtils.scala | 1 + .../shims/OffsetWindowFunctionMeta.scala | 1 + .../spark/rapids/shims/OrcCastingShims.scala | 1 + .../shims/OrcShims320untilAllBase.scala | 1 + .../spark/rapids/shims/PlanShimsImpl.scala | 1 + .../rapids/shims/RapidsCsvScanMeta.scala | 1 + .../spark/rapids/shims/RebaseShims.scala | 1 + .../rapids/shims/ShimAQEShuffleReadExec.scala | 1 + .../rapids/shims/ShimPredicateHelper.scala | 1 + .../rapids/shims/Spark320PlusShims.scala | 1 + .../shims/TagScanForRuntimeFiltering.scala | 1 + .../nvidia/spark/rapids/shims/TreeNode.scala | 1 + .../spark/rapids/shims/TypeSigUtil.scala | 1 + .../spark/rapids/shims/YearParseUtil.scala | 1 + .../spark/rapids/shims/gpuWindows.scala | 1 + .../shims/GpuShuffleBlockResolver.scala | 1 + .../rapids/shims/ShuffledBatchRDDUtil.scala | 1 + .../api/python/ShimBasePythonRunner.scala | 1 + .../shims/storage/ShimDiskBlockManager.scala | 1 + .../spark/sql/rapids/shims/AvroUtils.scala | 1 + .../shims/RapidsShuffleThreadedWriter.scala | 1 + .../sql/rapids/shims/Spark32XShimsUtils.scala | 1 + .../rapids/shims/datetimeExpressions.scala | 1 + .../storage/RapidsPushBasedFetchHelper.scala | 1 + .../RapidsShuffleBlockFetcherIterator.scala | 1 + .../rapids/shims/Spark321PlusShims.scala | 1 + .../nvidia/spark/rapids/shims/AQEUtils.scala | 1 + .../rapids/shims/AggregationTagging.scala | 1 + .../spark/rapids/shims/DeltaLakeUtils.scala | 1 + .../spark/rapids/shims/DistributionUtil.scala | 1 + .../rapids/shims/FileSourceScanExecMeta.scala | 1 + .../shims/GpuRunningWindowExecMeta.scala | 1 + .../rapids/shims/GpuWindowInPandasExec.scala | 1 + .../shims/PartitionedFileUtilsShim.scala | 1 + .../shims/ShimBroadcastExchangeLike.scala | 1 + .../spark/rapids/shims/ShimLeafExecNode.scala | 1 + .../rapids/shims/Spark321PlusDBShims.scala | 1 + .../shims/GpuSubqueryBroadcastMeta.scala | 1 + ...ReuseGpuBroadcastExchangeAndSubquery.scala | 1 + .../rapids/shims/GpuShuffleExchangeExec.scala | 1 + .../shims/GpuArrowPythonRunnerShims.scala | 1 + .../shims/GpuGroupUDFArrowPythonRunner.scala | 1 + .../nvidia/spark/rapids/shims/AnsiUtil.scala | 1 + .../spark/rapids/shims/CastCheckShims.scala | 1 + .../rapids/shims/CharVarcharUtilsShims.scala | 1 + .../rapids/shims/DayTimeIntervalShims.scala | 1 + .../spark/rapids/shims/GpuDataSourceRDD.scala | 1 + .../rapids/shims/GpuHashPartitioning.scala | 1 + .../spark/rapids/shims/GpuIntervalUtils.scala | 1 + .../rapids/shims/GpuRangePartitioning.scala | 1 + .../spark/rapids/shims/GpuTypeShims.scala | 1 + .../spark/rapids/shims/OrcReadingShims.scala | 1 + .../nvidia/spark/rapids/shims/OrcShims.scala | 1 + .../rapids/shims/ParquetFieldIdShims.scala | 1 + .../rapids/shims/ParquetSchemaClipShims.scala | 1 + .../shims/PythonMapInArrowExecShims.scala | 1 + .../shims/RapidsFileSourceMetaUtils.scala | 1 + .../rapids/shims/RapidsOrcScanMeta.scala | 1 + .../rapids/shims/RapidsParquetScanMeta.scala | 1 + .../spark/rapids/shims/RoundingShims.scala | 1 + .../spark/rapids/shims/ScanExecShims.scala | 1 + .../spark/sql/catalyst/csv/GpuCsvUtils.scala | 1 + .../sql/catalyst/json/GpuJsonUtils.scala | 1 + .../parquet/ShimCurrentBatchIterator.scala | 1 + .../shims/ShimVectorizedColumnReader.scala | 1 + .../shims/GpuPythonMapInArrowExec.scala | 1 + .../shims/RapidsErrorUtilsFor330plus.scala | 1 + .../shims/RapidsShuffleThreadedReader.scala | 1 + .../rapids/shims/intervalExpressions.scala | 1 + .../types/shims/PartitionValueCastShims.scala | 1 + .../spark/rapids/shims/AnsiCastShim.scala | 1 + .../rapids/shims/BatchScanExecMeta.scala | 1 + .../rapids/shims/CastingConfigShim.scala | 1 + .../shims/DecimalArithmeticOverrides.scala | 1 + .../spark/rapids/shims/GetMapValueMeta.scala | 1 + .../spark/rapids/shims/GpuBatchScanExec.scala | 1 + .../rapids/shims/GpuBroadcastJoinMeta.scala | 1 + .../rapids/shims/ParquetStringPredShims.scala | 1 + .../ShimFilePartitionReaderFactory.scala | 1 + .../rapids/shims/ShuffleOriginUtil.scala | 1 + .../spark/rapids/shims/SparkShims.scala | 1 + .../spark/rapids/shims/TypeUtilsShims.scala | 1 + .../spark330db/SparkShimServiceProvider.scala | 1 + .../spark330db/RapidsShuffleManager.scala | 1 + .../parquet/rapids/shims/ParquetCVShims.scala | 1 + .../rapids/DataSourceStrategyUtils.scala | 1 + .../spark/sql/rapids/AggregateFunctions.scala | 1 + .../apache/spark/sql/rapids/arithmetic.scala | 1 + .../execution/GpuBroadcastHashJoinExec.scala | 1 + .../GpuBroadcastNestedLoopJoinExec.scala | 1 + .../GpuExecutorBroadcastHelper.scala | 1 + .../sql/rapids/execution/GpuShuffleMeta.scala | 1 + .../rapids/execution/ShimTrampolineUtil.scala | 1 + .../sql/rapids/shims/RapidsErrorUtils.scala | 1 + .../shims/SparkDateTimeExceptionShims.scala | 1 + .../shims/SparkUpgradeExceptionShims.scala | 1 + .../RapidsShuffleInternalManager.scala | 1 + 114 files changed, 200 insertions(+), 6 deletions(-) diff --git a/jenkins/databricks/build.sh b/jenkins/databricks/build.sh index caeb93b838b..bfe7e3f99ef 100755 --- a/jenkins/databricks/build.sh +++ b/jenkins/databricks/build.sh @@ -131,7 +131,7 @@ set_jars_prefixes() PREFIX_MVN_TREE=${PREFIX_SPARK}--maven-trees PREFIX_WS_SP_MVN_HADOOP=${PREFIX_MVN_TREE}--${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]} - if [[ $BASE_SPARK_VERSION == "3.3.0" ]] + if [[ $BASE_SPARK_VERSION == "3.3.0" ]] || [[ $BASE_SPARK_VERSION == "3.3.2" ]] then #something like hadoop3 HADOOP_MAJOR_VERSION_NUM_STRING=$(echo "${sw_versions[HADOOP]}" | sed 's/\./\_/g' | cut -d _ -f 1) @@ -149,12 +149,33 @@ set_jars_prefixes() set_sw_versions() { case "$BASE_SPARK_VERSION" in + "3.3.2") + sw_versions[ARROW]="7.0.0" + sw_versions[AVRO]="1.11.0" + sw_versions[COMMONS_IO]="2.11.0" + sw_versions[COMMONS_LANG3]="3.12.0" + sw_versions[DB]="-0002" + sw_versions[FASTERXML_JACKSON_DATABIND]="2.13.4.2" + sw_versions[FASTERXML_JACKSON]="2.13.4" + sw_versions[HADOOP]="3.2" + sw_versions[HIVE_FULL]="2.3.9" + sw_versions[HIVESTORAGE_API]="2.8.1" + sw_versions[JAVAASSIST]="3.25.0-GA" + sw_versions[JSON4S_AST]="3.7.0-M11" + sw_versions[JSON4S_CORE]="3.7.0-M11" + sw_versions[KRYO]="4.0.2" + sw_versions[ORC]="1.7.6" + sw_versions[PARQUET]="1.12.3" + sw_versions[PROTOBUF]="2.6.1" + sw_versions[LOG4JCORE]="2.18.0" + ;; "3.3.0") sw_versions[ARROW]="7.0.0" sw_versions[AVRO]="1.11.0" sw_versions[COMMONS_IO]="2.11.0" sw_versions[COMMONS_LANG3]="3.12.0" sw_versions[DB]="-0007" + sw_versions[FASTERXML_JACKSON_DATABIND]="2.13.4" sw_versions[FASTERXML_JACKSON]="2.13.4" sw_versions[HADOOP]="3.2" sw_versions[HIVE_FULL]="2.3.9" @@ -174,6 +195,7 @@ set_sw_versions() sw_versions[COMMONS_IO]="2.8.0" sw_versions[COMMONS_LANG3]="3.12.0" sw_versions[DB]="-0007" + sw_versions[FASTERXML_JACKSON_DATABIND]="2.12.3" sw_versions[FASTERXML_JACKSON]="2.12.3" sw_versions[HADOOP]="3.2" sw_versions[HIVE_FULL]="2.3.9" @@ -212,10 +234,23 @@ set_dep_jars() dep_jars[ANNOT]=${PREFIX_SPARK}--common--tags--tags-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy.jar artifacts[CORE]="-DgroupId=org.apache.spark -DartifactId=spark-core_${SCALA_VERSION}" dep_jars[CORE]=${PREFIX_SPARK}--core--core-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy.jar + artifacts[HIVE]="-DgroupId=org.apache.spark -DartifactId=spark-hive_${SCALA_VERSION}" - dep_jars[HIVE]=${PREFIX_SPARK}--sql--hive--hive-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy_shaded.jar + if [[ "$BASE_SPARK_VERSION" == "3.3.2" ]]; then + # TODO what is significance of 819592503 and is this going to change over time? + dep_jars[HIVE]=${PREFIX_SPARK}--sql--hive--hive-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_shaded---819592503--hive-unshaded-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy.jar + else + dep_jars[HIVE]=${PREFIX_SPARK}--sql--hive--hive-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy_shaded.jar + fi + artifacts[HIVEEXEC]="-DgroupId=org.apache.hive -DartifactId=hive-exec" - dep_jars[HIVEEXEC]=${PREFIX_SPARK}--patched-hive-with-glue--hive-exec-core_shaded.jar + if [[ "$BASE_SPARK_VERSION" == "3.3.2" ]]; then + # TODO what is significance of 593920692 and is this going to change over time? + dep_jars[HIVEEXEC]=${PREFIX_SPARK}--patched-hive-with-glue--hive-exec_filtered---593920692--org.apache.hive__hive-exec-core__${sw_versions[HIVE_FULL]}.jar + else + dep_jars[HIVEEXEC]=${PREFIX_SPARK}--patched-hive-with-glue--hive-exec-core_shaded.jar + fi + artifacts[HIVESERDE]="-DgroupId=org.apache.hive -DartifactId=hive-serde" dep_jars[HIVESERDE]=${PREFIX_WS_SP_MVN_HADOOP}--org.apache.hive--hive-serde--org.apache.hive__hive-serde__${sw_versions[HIVE_FULL]}.jar artifacts[HIVESTORAGE]="-DgroupId=org.apache.hive -DartifactId=hive-storage-api" @@ -257,11 +292,18 @@ set_dep_jars() artifacts[JAVAASSIST]="-DgroupId=org.javaassist -DartifactId=javaassist" dep_jars[JAVAASSIST]=${PREFIX_WS_SP_MVN_HADOOP}--org.javassist--javassist--org.javassist__javassist__${sw_versions[JAVAASSIST]}.jar artifacts[JACKSONCORE]="-DgroupId=com.fasterxml.jackson.core -DartifactId=jackson-core" - dep_jars[JACKSONCORE]=${PREFIX_WS_SP_MVN_HADOOP}--com.fasterxml.jackson.core--jackson-databind--com.fasterxml.jackson.core__jackson-databind__${sw_versions[FASTERXML_JACKSON]}.jar + dep_jars[JACKSONCORE]=${PREFIX_WS_SP_MVN_HADOOP}--com.fasterxml.jackson.core--jackson-databind--com.fasterxml.jackson.core__jackson-databind__${sw_versions[FASTERXML_JACKSON_DATABIND]}.jar artifacts[JACKSONANNOTATION]="-DgroupId=com.fasterxml.jackson.core -DartifactId=jackson-annotations" dep_jars[JACKSONANNOTATION]=${PREFIX_WS_SP_MVN_HADOOP}--com.fasterxml.jackson.core--jackson-annotations--com.fasterxml.jackson.core__jackson-annotations__${sw_versions[FASTERXML_JACKSON]}.jar + artifacts[AVROSPARK]="-DgroupId=org.apache.spark -DartifactId=spark-avro_${SCALA_VERSION}" - dep_jars[AVROSPARK]=${PREFIX_SPARK}--vendor--avro--avro-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy_shaded.jar + if [[ "$BASE_SPARK_VERSION" == "3.3.2" ]]; then + # TODO what is significance of 1954496799 and is this going to change over time? + dep_jars[AVROSPARK]=${PREFIX_SPARK}--vendor--avro--avro-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_shaded---1954496799--avro-unshaded-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy.jar + else + dep_jars[AVROSPARK]=${PREFIX_SPARK}--vendor--avro--avro-${HIVE_VER_STRING}__hadoop-${sw_versions[HADOOP]}_${SCALA_VERSION}_deploy_shaded.jar + fi + artifacts[AVROMAPRED]="-DgroupId=org.apache.avro -DartifactId=avro-mapred" dep_jars[AVROMAPRED]=${PREFIX_WS_SP_MVN_HADOOP}--org.apache.avro--avro-mapred--org.apache.avro__avro-mapred__${sw_versions[AVRO]}.jar artifacts[AVRO]="-DgroupId=org.apache.avro -DartifactId=avro" diff --git a/pom.xml b/pom.xml index 7e235968fe8..d88fc60f015 100644 --- a/pom.xml +++ b/pom.xml @@ -530,6 +530,44 @@ aggregator + + + + release332db + + + buildver + 332db + + + + + 3.4.4 + spark332db + + ${spark332db.version} + ${spark332db.version} + 3.3.1 + true + 1.12.0 + ${spark330.iceberg.version} + + + delta-lake/delta-spark330db + dist + integration_tests + shuffle-plugin + sql-plugin + tests + udf-compiler + aggregator + + udf-compiler @@ -632,6 +670,7 @@ 3.4.0 3.3.0.3.3.7180.0-274 3.3.0-databricks + 3.3.2-databricks 3.12.4 4.3.0 3.2.0 @@ -678,7 +717,8 @@ 321db, - 330db + 330db, + 332db + + 4.0.0 + + + com.nvidia + rapids-4-spark-parent + 23.06.0-SNAPSHOT + ../../pom.xml + + + rapids-4-spark-delta-spark332db_2.12 + RAPIDS Accelerator for Apache Spark Databricks 12.2 Delta Lake Support + Databricks 12.2 Delta Lake support for the RAPIDS Accelerator for Apache Spark + 23.06.0-SNAPSHOT + + + + com.nvidia + rapids-4-spark-sql_${scala.binary.version} + ${project.version} + ${spark.version.classifier} + provided + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-annotation_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-network-common_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-network-shuffle_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-launcher_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-unsafe_${scala.binary.version} + ${spark.version} + provided + + + org.apache.avro + avro-mapred + ${spark.version} + provided + + + org.apache.avro + avro + ${spark.version} + provided + + + org.apache.hive + hive-exec + ${spark.version} + provided + + + org.apache.hive + hive-serde + ${spark.version} + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + + + com.fasterxml.jackson.core + jackson-core + ${spark.version} + provided + + + com.fasterxml.jackson.core + jackson-annotations + ${spark.version} + provided + + + org.json4s + json4s-ast_${scala.binary.version} + ${spark.version} + provided + + + org.json4s + json4s-core_${scala.binary.version} + ${spark.version} + provided + + + org.apache.commons + commons-io + ${spark.version} + provided + + + org.scala-lang + scala-reflect + ${scala.version} + provided + + + org.apache.commons + commons-lang3 + ${spark.version} + provided + + + com.esotericsoftware.kryo + kryo-shaded-db + ${spark.version} + provided + + + org.apache.parquet + parquet-hadoop + ${spark.version} + provided + + + org.apache.parquet + parquet-common + ${spark.version} + provided + + + org.apache.parquet + parquet-column + ${spark.version} + provided + + + org.apache.parquet + parquet-format + ${spark.version} + provided + + + org.apache.arrow + arrow-memory + ${spark.version} + provided + + + org.apache.arrow + arrow-vector + ${spark.version} + provided + + + org.apache.hadoop + hadoop-client + ${hadoop.client.version} + provided + + + org.apache.orc + orc-core + ${spark.version} + provided + + + org.apache.orc + orc-shims + ${spark.version} + provided + + + org.apache.orc + orc-mapreduce + ${spark.version} + provided + + + org.apache.hive + hive-storage-api + ${spark.version} + provided + + + com.google.protobuf + protobuf-java + ${spark.version} + 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 + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + false + + ${spark.version.classifier} + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.rat + apache-rat-plugin + + + + diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeleteCommand.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeleteCommand.scala new file mode 100644 index 00000000000..842266fd387 --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeleteCommand.scala @@ -0,0 +1,353 @@ +/* + * 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.actions.{Action, AddCDCFile, FileAction} +import com.databricks.sql.transaction.tahoe.commands.{DeleteCommandMetrics, DeleteMetric, DeltaCommand} +import com.databricks.sql.transaction.tahoe.commands.MergeIntoCommand.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.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)()) + + override lazy val metrics = createMetrics + + final override def run(sparkSession: SparkSession): Seq[Row] = { + val deltaLog = gpuDeltaLog.deltaLog + recordDeltaOperation(gpuDeltaLog.deltaLog, "delta.dml.delete") { + //TODO this method no longer exists + // deltaLog.assertRemovable() + gpuDeltaLog.withNewTransaction { txn => + val deleteActions = performDelete(sparkSession, deltaLog, txn) + if (deleteActions.nonEmpty) { + txn.commit(deleteActions, DeltaOperations.Delete(condition.map(_.sql).toSeq)) + } + } + // 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): Seq[Action] = { + 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[Action] = 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("numBytesAdded").set(numBytesAdded) + metrics("numBytesRemoved").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) + 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) + ) + + deleteActions + } + + /** + * 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-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDoAutoCompaction.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDoAutoCompaction.scala new file mode 100644 index 00000000000..9726511ad44 --- /dev/null +++ b/delta-lake/delta-spark332db/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-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala new file mode 100644 index 00000000000..6d5fdd6b1ff --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala @@ -0,0 +1,1172 @@ +/* + * 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, DeltaMergeIntoMatchedDeleteClause, DeltaMergeIntoNotMatchedInsertClause, DeltaMergeIntoMatchedClause, DeltaMergeIntoNotMatchedClause, DeltaMergeIntoMatchedUpdateClause, LogicalPlan, Project} +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[DeltaMergeIntoNotMatchedInsertClause], + 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], + 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.sql), + matchedClauses.map(DeltaOperations.MergePredicate(_)), + notMatchedClauses.map(DeltaOperations.MergePredicate(_)), + // TODO do we need to pass something in here? + 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) + 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, + 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]]], + 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 noopCopyOutputMetas = noopCopyOutput.map(wrap) + val deleteRowOutputMetas = deleteRowOutput.map(wrap) + val allMetas = Seq(targetRowHasNoMatchMeta, sourceRowHasNoMatchMeta) ++ + matchedConditionsMetas ++ matchedOutputsMetas.flatten.flatten ++ + notMatchedConditionsMetas ++ notMatchedOutputsMetas.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, + outputRowSchema.toAttributes, + targetRowHasNoMatch = targetRowHasNoMatch, + sourceRowHasNoMatch = sourceRowHasNoMatch, + matchedConditions = matchedConditions, + matchedOutputs = matchedOutputs, + notMatchedConditions = notMatchedConditions, + notMatchedOutputs = notMatchedOutputs, + noopCopyOutput = noopCopyOutput, + deleteRowOutput = deleteRowOutput) + Dataset.ofRows(spark, processedJoinPlan) + } else { + val joinedRowEncoder = RowEncoder(joinedPlan.schema) + val outputRowEncoder = RowEncoder(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.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-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala new file mode 100644 index 00000000000..0e8c7f74fde --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -0,0 +1,301 @@ +/* + * Copyright (c) 2022-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 ai.rapids.cudf.ColumnView +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.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 statsCollection = new GpuStatisticsCollection { + 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 batchStatsToRow = (columnViews: Array[ColumnView], row: InternalRow) => { + GpuStatisticsCollection.batchStatsToRow(statsSchema, columnViews, 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) = + normalizeData(deltaLog, 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() + + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( + new SerializableConfiguration(deltaLog.newDeltaHadoopConf()), + BasicWriteJobStatsTracker.metrics) + registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + statsTrackers.append(basicWriteJobStatsTracker) + gpuRapidsWrite.foreach { grw => + val hadoopConf = new SerializableConfiguration(spark.sparkContext.hadoopConfiguration) + val tracker = new GpuWriteJobStatsTracker(hadoopConf, 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 gpuFileFormat = deltaLog.fileFormat(metadata) match { + case _: DeltaParquetFileFormat => new GpuParquetFileFormat + case f => throw new IllegalStateException(s"file format $f is not supported") + } + + try { + logDebug(s"Physical plan for write:\n$physicalPlan") + GpuFileFormatWriter.write( + sparkSession = spark, + plan = physicalPlan, + fileFormat = gpuFileFormat, + committer = committer, + outputSpec = outputSpec, + // scalastyle:off deltahadoopconfiguration + hadoopConf = + spark.sessionState.newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options), + // scalastyle:on deltahadoopconfiguration + 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-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimizeExecutor.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimizeExecutor.scala new file mode 100644 index 00000000000..cfa1468b7c9 --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimizeExecutor.scala @@ -0,0 +1,405 @@ +/* + * 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.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.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.map(_.sql), 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) + } + } + + /** + * 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, + zOrderByColumns) + } 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() + 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))) + } +} \ No newline at end of file diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuUpdateCommand.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuUpdateCommand.scala new file mode 100644 index 00000000000..96d55f34135 --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuUpdateCommand.scala @@ -0,0 +1,270 @@ +/* + * 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.actions.{AddCDCFile, AddFile, FileAction} +import com.databricks.sql.transaction.tahoe.commands.{DeltaCommand, 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)") + ) + + final override def run(sparkSession: SparkSession): Seq[Row] = { + recordDeltaOperation(tahoeFileIndex.deltaLog, "delta.dml.update") { + val deltaLog = tahoeFileIndex.deltaLog + + //TODO this method no longer exists + //deltaLog.assertRemovable() + + gpuDeltaLog.withNewTransaction { txn => + 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) + } + txn.registerSQLMetrics(sparkSession, metrics) + txn.commit(totalActions, DeltaOperations.Update(condition.map(_.toString))) + // 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) + ) + } + + /** + * 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, + 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-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/InvariantViolationExceptionShim.scala b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/InvariantViolationExceptionShim.scala new file mode 100644 index 00000000000..8d8a48157c6 --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/InvariantViolationExceptionShim.scala @@ -0,0 +1,30 @@ +/* + * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.nvidia.spark.rapids.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-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/ShimDeltaUDF.scala b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/ShimDeltaUDF.scala new file mode 100644 index 00000000000..fd9052d9691 --- /dev/null +++ b/delta-lake/delta-spark332db/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/pom.xml b/pom.xml index 22a7dcb3b56..7fdac2918fa 100644 --- a/pom.xml +++ b/pom.xml @@ -583,7 +583,7 @@ ${spark330.iceberg.version} - delta-lake/delta-spark330db + delta-lake/delta-spark332db dist integration_tests shuffle-plugin diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index 89d8afbd0e1..a8f054e60c2 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -30,7 +30,6 @@ {"spark": "330db"} {"spark": "331"} {"spark": "332"} -{"spark": "332db"} {"spark": "333"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims From fe590ba54223e7c81b71a94b1030d85caeeffa3f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 15 May 2023 16:57:03 -0600 Subject: [PATCH 24/68] fix another compilation issue --- .../sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala index 6d5fdd6b1ff..6e971a58172 100644 --- a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuMergeIntoCommand.scala @@ -44,7 +44,7 @@ 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, DeltaMergeIntoMatchedDeleteClause, DeltaMergeIntoNotMatchedInsertClause, DeltaMergeIntoMatchedClause, DeltaMergeIntoNotMatchedClause, DeltaMergeIntoMatchedUpdateClause, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{DeltaMergeIntoClause, DeltaMergeIntoMatchedDeleteClause, DeltaMergeIntoMatchedClause, DeltaMergeIntoNotMatchedClause, DeltaMergeIntoMatchedUpdateClause, LogicalPlan, Project} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.LeafRunnableCommand @@ -133,7 +133,7 @@ object GpuMergeStats { metrics: Map[String, SQLMetric], condition: Expression, matchedClauses: Seq[DeltaMergeIntoMatchedClause], - notMatchedClauses: Seq[DeltaMergeIntoNotMatchedInsertClause], + notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], isPartitioned: Boolean): GpuMergeStats = { def metricValueIfPartitioned(metricName: String): Option[Long] = { From d96826bb90d6705e505e1fc4d9c828c0b0d2462e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 16 May 2023 11:46:21 -0600 Subject: [PATCH 25/68] Add DeltaLogShim --- .../spark/rapids/delta/RapidsDeltaUtils.scala | 5 +-- .../rapids/delta/shims/DeltaLogShim.scala | 32 +++++++++++++++++++ .../rapids/delta/shims/DeltaLogShim.scala | 32 +++++++++++++++++++ .../rapids/delta/shims/DeltaLogShim.scala | 31 ++++++++++++++++++ 4 files changed, 98 insertions(+), 2 deletions(-) create mode 100644 delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala create mode 100644 delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala create mode 100644 delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala index 99f110acb88..32d3bd1215c 100644 --- a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala @@ -18,6 +18,7 @@ package com.nvidia.spark.rapids.delta import com.databricks.sql.transaction.tahoe.{DeltaConfigs, DeltaLog, DeltaOptions, DeltaParquetFileFormat} import com.nvidia.spark.rapids.{DeltaFormatType, FileFormatChecks, GpuOverrides, GpuParquetFileFormat, RapidsMeta, TypeSig, WriteFileOp} +import com.nvidia.spark.rapids.delta.shims.DeltaLogShim import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.DataSourceUtils @@ -32,7 +33,7 @@ object RapidsDeltaUtils { options: Map[String, String], spark: SparkSession): Unit = { FileFormatChecks.tag(meta, schema, DeltaFormatType, WriteFileOp) - deltaLog.fileFormat() match { + DeltaLogShim.fileFormat(deltaLog) match { case _: DeltaParquetFileFormat => GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) case f => @@ -65,7 +66,7 @@ object RapidsDeltaUtils { orderableTypeSig.isSupportedByPlugin(t) } if (unorderableTypes.nonEmpty) { - val metadata = deltaLog.snapshot.metadata + val metadata = DeltaLogShim.getMetadata(deltaLog) val hasPartitioning = metadata.partitionColumns.nonEmpty || options.get(DataSourceUtils.PARTITIONING_COLUMNS_KEY).exists(_.nonEmpty) if (!hasPartitioning) { diff --git a/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala b/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala new file mode 100644 index 00000000000..427ab1d64ec --- /dev/null +++ b/delta-lake/delta-spark321db/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() + } + + def getMetadata(deltaLog: DeltaLog): Metadata = { + deltaLog.snapshot.metadata + } +} diff --git a/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala b/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala new file mode 100644 index 00000000000..427ab1d64ec --- /dev/null +++ b/delta-lake/delta-spark330db/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() + } + + def getMetadata(deltaLog: DeltaLog): Metadata = { + deltaLog.snapshot.metadata + } +} diff --git a/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala new file mode 100644 index 00000000000..9cf80f110bf --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/shims/DeltaLogShim.scala @@ -0,0 +1,31 @@ +/* + * 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.metadata) + } + def getMetadata(deltaLog: DeltaLog): Metadata = { + deltaLog.unsafeVolatileSnapshot.metadata + } +} From f307dafbdda31f93f6f8227156fe8f02e8b85101 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 16 May 2023 11:58:08 -0600 Subject: [PATCH 26/68] add 332db profile to aggregator --- aggregator/pom.xml | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/aggregator/pom.xml b/aggregator/pom.xml index 24f244fb4a8..a854e509ba2 100644 --- a/aggregator/pom.xml +++ b/aggregator/pom.xml @@ -505,6 +505,23 @@
+ + release332db + + + buildver + 332db + + + + + com.nvidia + rapids-4-spark-delta-spark332db_${scala.binary.version} + ${project.version} + ${spark.version.classifier} + + + release333 From bcffceaacfb14d5f5b8fec3d81f39c3694ec7e2f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 16 May 2023 15:35:48 -0600 Subject: [PATCH 27/68] trying to get integration tests running --- jenkins/databricks/build.sh | 2 +- jenkins/databricks/test.sh | 4 ++++ sql-plugin/pom.xml | 17 +++++++++++++++++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/jenkins/databricks/build.sh b/jenkins/databricks/build.sh index bfe7e3f99ef..4b40a97c02b 100755 --- a/jenkins/databricks/build.sh +++ b/jenkins/databricks/build.sh @@ -310,7 +310,7 @@ set_dep_jars() dep_jars[AVRO]=${PREFIX_WS_SP_MVN_HADOOP}--org.apache.avro--avro--org.apache.avro__avro__${sw_versions[AVRO]}.jar # log4j-core - if [[ "$BASE_SPARK_VERSION" == "3.3.0" ]]; then + if [[ "$BASE_SPARK_VERSION" == "3.3.0" ]] || [[ "$BASE_SPARK_VERSION" == "3.3.2" ]]; then artifacts[LOG4JCORE]="-DgroupId=org.apache.logging.log4j -DartifactId=log4j-core" dep_jars[LOG4JCORE]=${PREFIX_WS_SP_MVN_HADOOP}--org.apache.logging.log4j--log4j-core--org.apache.logging.log4j__log4j-core__${sw_versions[LOG4JCORE]}.jar fi diff --git a/jenkins/databricks/test.sh b/jenkins/databricks/test.sh index c8a8cff7633..1e9a2e6faa8 100755 --- a/jenkins/databricks/test.sh +++ b/jenkins/databricks/test.sh @@ -82,6 +82,10 @@ case "$BASE_SPARK_VERSION" in # Available versions https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.3_2.12/ sw_versions[ICEBERG]=${ICEBERG_VERSION:-'0.14.1'} ;; + "3.3.2") + # Available versions https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.3_2.12/ + sw_versions[ICEBERG]=${ICEBERG_VERSION:-'0.14.1'} + ;; "3.2.1") # Available versions https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.2_2.12/ sw_versions[ICEBERG]=${ICEBERG_VERSION:-'0.13.2'} diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 9465ef7eb35..0d6a1bb1c1c 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -223,6 +223,23 @@ + + release332db + + + buildver + 332db + + + + + org.apache.logging.log4j + log4j-core + ${spark.version} + provided + + +