Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add DeltaLake support without Deletion Vectors for Databricks 14.3 [databricks] #12048

Open
wants to merge 32 commits into
base: branch-25.02
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 16 commits
Commits
Show all changes
32 commits
Select commit Hold shift + click to select a range
36aab4b
Added delta-lake support for Databricks 14.3
razajafri Jan 28, 2025
ec3b55d
xfailed delta_lake_delete_test.py due to lacking deletion vector writ…
razajafri Jan 29, 2025
9cd136c
skip low_shuffle_merge_test for any databricks version besides 13.3
razajafri Jan 29, 2025
4b23ccd
added Execs to run on CPU for auto_compact tests
razajafri Jan 29, 2025
fe85f65
xfailed delta_lake_merge_test.py
razajafri Jan 29, 2025
fcce0a8
xfailed delta_lake_test.py
razajafri Jan 29, 2025
e5b2ff0
xfailed delta_lake_update_test.py
razajafri Jan 30, 2025
e482937
reverted change
razajafri Jan 30, 2025
4d98a62
xfailed and fixed some failing tests in delta_lake_write_test.py
razajafri Jan 30, 2025
f14ff42
xfailed delta_zorder_test.py
razajafri Jan 30, 2025
9c908f7
Signing off
razajafri Jan 30, 2025
6cfce00
updated copyrights and fixed line length
razajafri Jan 30, 2025
c6f9199
Removed multiple versions of DatabricksDeltaProviderBase
razajafri Jan 30, 2025
1c77a22
reverted test.sh changes
razajafri Jan 30, 2025
7360dd3
updated copyrights
razajafri Jan 30, 2025
5b3aaf2
removed fastparquet.txt
razajafri Jan 30, 2025
c41d731
Added copyrights from Delta-io project
razajafri Jan 31, 2025
013a794
Modified the logic for turning deletion vectors on/off to use the
razajafri Jan 31, 2025
bf90960
renamed parameter names
razajafri Jan 31, 2025
df583cb
Added DeltaInvariantCheckerExec to fallback
razajafri Jan 31, 2025
70edea2
addressed review comments
razajafri Jan 31, 2025
5b3acd9
Update copyrights on DeltaProbe.scala
razajafri Jan 31, 2025
2223cd8
Addressed review comments
razajafri Jan 31, 2025
f77ef04
Improved formatting
razajafri Feb 1, 2025
f7390e3
Merge remote-tracking branch 'origin/branch-25.02' into HEAD
razajafri Feb 1, 2025
88457fc
Removed DeltaInvariant from the allow_non_gpu as it's included from t…
razajafri Feb 1, 2025
3ef1f17
Disabling deletion vector using the conf as setting the tblproperties…
razajafri Feb 1, 2025
6f47d0e
hard coding disabling deletion vectors as it doesn't matter in low_sh…
razajafri Feb 1, 2025
1a8b0f9
added fallback tests
razajafri Feb 3, 2025
6e2175f
Fixed unclosed paranthesis
razajafri Feb 4, 2025
c235956
Add dv tblproperties only for specific versions of delta-lake
razajafri Feb 5, 2025
4a97097
fixed syntax error
razajafri Feb 5, 2025
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2023, NVIDIA CORPORATION.
* Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand Down Expand Up @@ -81,6 +81,24 @@ trait GpuDeltaCatalogBase extends StagingTableCatalog {
new GpuStagedDeltaTableV2(ident, schema, partitions, properties, operation)
}

protected def getWriter(sourceQuery: Option[DataFrame],
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method is overridden in Databricks 14.3's version of GpuDeltaCatalog

path: Path,
comment: Option[String],
schema: Option[StructType],
saveMode: SaveMode,
catalogTable: CatalogTable): Option[LogicalPlan] = {
sourceQuery.map { df =>
WriteIntoDelta(
DeltaLog.forTable(spark, path),
saveMode,
new DeltaOptions(catalogTable.storage.properties, spark.sessionState.conf),
catalogTable.partitionColumnNames,
catalogTable.properties ++ comment.map("comment" -> _),
df,
schemaInCatalog = schema)
}
}

/**
* Creates a Delta table using GPU for writing the data
*
Expand Down Expand Up @@ -164,16 +182,9 @@ trait GpuDeltaCatalogBase extends StagingTableCatalog {

val withDb = verifyTableAndSolidify(tableDesc, None)

val writer = sourceQuery.map { df =>
WriteIntoDelta(
DeltaLog.forTable(spark, new Path(loc)),
operation.mode,
new DeltaOptions(withDb.storage.properties, spark.sessionState.conf),
withDb.partitionColumnNames,
withDb.properties ++ commentOpt.map("comment" -> _),
df,
schemaInCatalog = if (newSchema != schema) Some(newSchema) else None)
}
val schemaInCatalog = if (newSchema != schema) Some(newSchema) else None
val writer = getWriter(sourceQuery, new Path(loc), commentOpt, schemaInCatalog,
operation.mode, withDb)

val gpuCreateTableCommand = buildGpuCreateDeltaTableCommand(
rapidsConf,
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2022-2024, NVIDIA CORPORATION.
* Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -22,25 +22,24 @@ import scala.collection.JavaConverters.mapAsScalaMapConverter
import scala.collection.mutable

import com.databricks.sql.managedcatalog.UnityCatalogV2Proxy
import com.databricks.sql.transaction.tahoe.{DeltaLog, DeltaOptions, DeltaParquetFileFormat}
import com.databricks.sql.transaction.tahoe.{DeltaLog, DeltaParquetFileFormat}
import com.databricks.sql.transaction.tahoe.catalog.{DeltaCatalog, DeltaTableV2}
import com.databricks.sql.transaction.tahoe.commands.{DeleteCommand, DeleteCommandEdge, MergeIntoCommand, MergeIntoCommandEdge, UpdateCommand, UpdateCommandEdge, WriteIntoDelta}
import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaLog, GpuDeltaSupportsWrite, GpuDeltaV1Write, GpuWriteIntoDelta}
import com.databricks.sql.transaction.tahoe.commands.{DeleteCommand, DeleteCommandEdge, MergeIntoCommand, MergeIntoCommandEdge, UpdateCommand, UpdateCommandEdge}
import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaSupportsWrite, GpuDeltaV1Write}
import com.databricks.sql.transaction.tahoe.sources.{DeltaDataSource, DeltaSourceUtils}
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.delta.shims.DeltaLogShim
import org.apache.hadoop.fs.Path

import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, SupportsWrite}
import org.apache.spark.sql.connector.write.V1Write
import org.apache.spark.sql.execution.FileSourceScanExec
import org.apache.spark.sql.execution.command.RunnableCommand
import org.apache.spark.sql.execution.datasources.{FileFormat, LogicalRelation, SaveIntoDataSourceCommand}
import org.apache.spark.sql.execution.datasources.{FileFormat, SaveIntoDataSourceCommand}
import org.apache.spark.sql.execution.datasources.v2.{AppendDataExecV1, AtomicCreateTableAsSelectExec, AtomicReplaceTableAsSelectExec, OverwriteByExpressionExecV1}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids.ExternalSource
import org.apache.spark.sql.sources.{CreatableRelationProvider, InsertableRelation}
import org.apache.spark.sql.sources.CreatableRelationProvider
import org.apache.spark.sql.util.CaseInsensitiveStringMap

/**
Expand Down Expand Up @@ -161,7 +160,7 @@ trait DatabricksDeltaProviderBase extends DeltaProviderImplBase {
getWriteOptions(cpuExec.writeOptions), cpuExec.session)
}

private case class DeltaWriteV1Config(
protected case class DeltaWriteV1Config(
deltaLog: DeltaLog,
forceOverwrite: Boolean,
options: mutable.HashMap[String, String])
Expand Down Expand Up @@ -286,35 +285,9 @@ trait DatabricksDeltaProviderBase extends DeltaProviderImplBase {
GpuOverwriteByExpressionExecV1(cpuExec.table, cpuExec.plan, cpuExec.refreshCache, gpuWrite)
}

private def toGpuWrite(
writeConfig: DeltaWriteV1Config,
rapidsConf: RapidsConf): V1Write = new GpuDeltaV1Write {
override def toInsertableRelation(): InsertableRelation = {
new InsertableRelation {
override def insert(data: DataFrame, overwrite: Boolean): Unit = {
val session = data.sparkSession
val deltaLog = writeConfig.deltaLog

// TODO: Get the config from WriteIntoDelta's txn.
val cpuWrite = WriteIntoDelta(
deltaLog,
if (writeConfig.forceOverwrite) SaveMode.Overwrite else SaveMode.Append,
new DeltaOptions(writeConfig.options.toMap, session.sessionState.conf),
Nil,
DeltaLogShim.getMetadata(deltaLog).configuration,
data)
val gpuWrite = GpuWriteIntoDelta(new GpuDeltaLog(deltaLog, rapidsConf), cpuWrite)
gpuWrite.run(session)

// TODO: Push this to Apache Spark
// Re-cache all cached plans(including this relation itself, if it's cached) that refer
// to this data source relation. This is the behavior for InsertInto
session.sharedState.cacheManager.recacheByPlan(
session, LogicalRelation(deltaLog.createRelation()))
}
}
}
}
protected def toGpuWrite(
writeConfig: DeltaWriteV1Config,
rapidsConf: RapidsConf): V1Write
}

class DeltaCreatableRelationProviderMeta(
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2023, NVIDIA CORPORATION.
* Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -16,13 +16,50 @@

package com.nvidia.spark.rapids.delta

import com.databricks.sql.transaction.tahoe.rapids.GpuDeltaCatalog
import com.nvidia.spark.rapids.{AtomicCreateTableAsSelectExecMeta, AtomicReplaceTableAsSelectExecMeta, GpuExec}
import com.databricks.sql.transaction.tahoe.DeltaOptions
import com.databricks.sql.transaction.tahoe.commands.WriteIntoDelta
import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaCatalog, GpuDeltaLog, GpuDeltaV1Write, GpuWriteIntoDelta}
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.delta.shims.DeltaLogShim

import org.apache.spark.sql.{DataFrame, SaveMode}
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.connector.write.V1Write
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.datasources.v2.{AtomicCreateTableAsSelectExec, AtomicReplaceTableAsSelectExec}
import org.apache.spark.sql.execution.datasources.v2.rapids.{GpuAtomicCreateTableAsSelectExec, GpuAtomicReplaceTableAsSelectExec}
import org.apache.spark.sql.sources.InsertableRelation

object DeltaSpark330DBProvider extends DatabricksDeltaProviderBase {
override protected def toGpuWrite(
writeConfig: DeltaWriteV1Config,
rapidsConf: RapidsConf): V1Write = new GpuDeltaV1Write {
override def toInsertableRelation(): InsertableRelation = {
new InsertableRelation {
override def insert(data: DataFrame, overwrite: Boolean): Unit = {
val session = data.sparkSession
val deltaLog = writeConfig.deltaLog

// TODO: Get the config from WriteIntoDelta's txn.
val cpuWrite = WriteIntoDelta(
deltaLog,
if (writeConfig.forceOverwrite) SaveMode.Overwrite else SaveMode.Append,
new DeltaOptions(writeConfig.options.toMap, session.sessionState.conf),
Nil,
DeltaLogShim.getMetadata(deltaLog).configuration,
data)
val gpuWrite = GpuWriteIntoDelta(new GpuDeltaLog(deltaLog, rapidsConf), cpuWrite)
gpuWrite.run(session)

// TODO: Push this to Apache Spark
// Re-cache all cached plans(including this relation itself, if it's cached) that refer
// to this data source relation. This is the behavior for InsertInto
session.sharedState.cacheManager.recacheByPlan(
session, LogicalRelation(deltaLog.createRelation()))
}
}
}
}

override def convertToGpu(
cpuExec: AtomicCreateTableAsSelectExec,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This file is the same in every version of Databricks except for Databricks 14.3

* Copyright (c) 2022-2025, NVIDIA CORPORATION.
mythrocks marked this conversation as resolved.
Show resolved Hide resolved
*
* This file was derived from DeltaDataSource.scala in the
* Delta Lake project at https://github.com/delta-io/delta.
*
* Copyright (2021) The Delta Lake Project Authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.nvidia.spark.rapids.delta

import com.databricks.sql.transaction.tahoe.{DeltaConfigs, DeltaErrors, DeltaOptions}
import com.databricks.sql.transaction.tahoe.commands.WriteIntoDelta
import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaLog, GpuWriteIntoDelta}
import com.databricks.sql.transaction.tahoe.sources.{DeltaDataSource, DeltaSourceUtils}
import com.nvidia.spark.rapids.{GpuCreatableRelationProvider, RapidsConf}

import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
import org.apache.spark.sql.sources.BaseRelation

/** GPU version of DeltaDataSource from Delta Lake. */
class GpuDeltaDataSource(rapidsConf: RapidsConf) extends GpuCreatableRelationProvider {
override def createRelation(
sqlContext: SQLContext,
mode: SaveMode,
parameters: Map[String, String],
data: DataFrame): BaseRelation = {
val path = parameters.getOrElse("path", {
throw DeltaErrors.pathNotSpecifiedException
})
val partitionColumns = parameters.get(DeltaSourceUtils.PARTITIONING_COLUMNS_KEY)
.map(DeltaDataSource.decodePartitioningColumns)
.getOrElse(Nil)

val gpuDeltaLog = GpuDeltaLog.forTable(sqlContext.sparkSession, path, parameters, rapidsConf)
GpuWriteIntoDelta(
gpuDeltaLog,
WriteIntoDelta(
deltaLog = gpuDeltaLog.deltaLog,
mode = mode,
new DeltaOptions(parameters, sqlContext.sparkSession.sessionState.conf),
partitionColumns = partitionColumns,
configuration = DeltaConfigs.validateConfigurations(
parameters.filterKeys(_.startsWith("delta.")).toMap),
data = data)).run(sqlContext.sparkSession)

gpuDeltaLog.deltaLog.createRelation()
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This file is the same in every version of Databricks except for 14.3

* Copyright (c) 2022-2025, NVIDIA CORPORATION.
*
* This file was derived from WriteIntoDelta.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.{DeltaOperations, OptimisticTransaction}
import com.databricks.sql.transaction.tahoe.commands.WriteIntoDelta

import org.apache.spark.sql._
import org.apache.spark.sql.execution.command.LeafRunnableCommand

/** GPU version of Delta Lake's WriteIntoDelta. */
case class GpuWriteIntoDelta(
gpuDeltaLog: GpuDeltaLog,
cpuWrite: WriteIntoDelta)
extends LeafRunnableCommand {

override def run(sparkSession: SparkSession): Seq[Row] = {
gpuDeltaLog.withNewTransaction { txn =>
// If this batch has already been executed within this query, then return.
val skipExecution = hasBeenExecuted(txn)
if (skipExecution) {
return Seq.empty
}

val actions = cpuWrite.write(txn, sparkSession)
val operation = DeltaOperations.Write(
cpuWrite.mode,
Option(cpuWrite.partitionColumns),
cpuWrite.options.replaceWhere,
cpuWrite.options.userMetadata)
txn.commit(actions, operation)
}
Seq.empty
}

/**
* Returns true if there is information in the spark session that indicates that this write, which
* is part of a streaming query and a batch, has already been successfully written.
*/
private def hasBeenExecuted(txn: OptimisticTransaction): Boolean = {
val txnVersion = cpuWrite.options.txnVersion
val txnAppId = cpuWrite.options.txnAppId
for (v <- txnVersion; a <- txnAppId) {
val currentVersion = txn.txnVersion(a)
if (currentVersion >= v) {
logInfo(s"Transaction write of version $v for application id $a " +
s"has already been committed in Delta table id ${txn.deltaLog.tableId}. " +
s"Skipping this write.")
return true
}
}
false
}
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2023, NVIDIA CORPORATION.
* Copyright (c) 2023-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -16,13 +16,50 @@

package com.nvidia.spark.rapids.delta

import com.databricks.sql.transaction.tahoe.rapids.GpuDeltaCatalog
import com.nvidia.spark.rapids.{AtomicCreateTableAsSelectExecMeta, AtomicReplaceTableAsSelectExecMeta, GpuExec}
import com.databricks.sql.transaction.tahoe.DeltaOptions
import com.databricks.sql.transaction.tahoe.commands.WriteIntoDelta
import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaCatalog, GpuDeltaLog, GpuDeltaV1Write, GpuWriteIntoDelta}
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.delta.shims.DeltaLogShim

import org.apache.spark.sql.{DataFrame, SaveMode}
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.connector.write.V1Write
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.datasources.v2.{AtomicCreateTableAsSelectExec, AtomicReplaceTableAsSelectExec}
import org.apache.spark.sql.execution.datasources.v2.rapids.{GpuAtomicCreateTableAsSelectExec, GpuAtomicReplaceTableAsSelectExec}
import org.apache.spark.sql.sources.InsertableRelation

object DeltaSpark332DBProvider extends DatabricksDeltaProviderBase {
override protected def toGpuWrite(
writeConfig: DeltaWriteV1Config,
rapidsConf: RapidsConf): V1Write = new GpuDeltaV1Write {
override def toInsertableRelation(): InsertableRelation = {
new InsertableRelation {
override def insert(data: DataFrame, overwrite: Boolean): Unit = {
val session = data.sparkSession
val deltaLog = writeConfig.deltaLog

// TODO: Get the config from WriteIntoDelta's txn.
val cpuWrite = WriteIntoDelta(
deltaLog,
if (writeConfig.forceOverwrite) SaveMode.Overwrite else SaveMode.Append,
new DeltaOptions(writeConfig.options.toMap, session.sessionState.conf),
Nil,
DeltaLogShim.getMetadata(deltaLog).configuration,
data)
val gpuWrite = GpuWriteIntoDelta(new GpuDeltaLog(deltaLog, rapidsConf), cpuWrite)
gpuWrite.run(session)

// TODO: Push this to Apache Spark
// Re-cache all cached plans(including this relation itself, if it's cached) that refer
// to this data source relation. This is the behavior for InsertInto
session.sharedState.cacheManager.recacheByPlan(
session, LogicalRelation(deltaLog.createRelation()))
}
}
}
}

override def convertToGpu(
cpuExec: AtomicCreateTableAsSelectExec,
Expand Down
Loading
Loading