-
Notifications
You must be signed in to change notification settings - Fork 242
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Moved common code to specific folders
- Loading branch information
Showing
12 changed files
with
1,446 additions
and
0 deletions.
There are no files selected for viewing
61 changes: 61 additions & 0 deletions
61
...330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuDeltaDataSource.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,61 @@ | ||
/* | ||
* Copyright (c) 2022-2023, NVIDIA CORPORATION. | ||
* | ||
* This file was derived from DeltaDataSource.scala in the | ||
* Delta Lake project at https://github.com/delta-io/delta. | ||
* | ||
* Copyright (2021) The Delta Lake Project Authors. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package com.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() | ||
} | ||
} |
73 changes: 73 additions & 0 deletions
73
...k330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuWriteIntoDelta.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/* | ||
* Copyright (c) 2022-2023, 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 | ||
} | ||
} |
Oops, something went wrong.