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

Support auto-compaction for Delta tables on [databricks] #7889

Merged
merged 18 commits into from
Mar 27, 2023
Merged
Show file tree
Hide file tree
Changes from 11 commits
Commits
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
Expand Up @@ -53,7 +53,7 @@ import org.apache.spark.util.Clock
* @param rapidsConf RAPIDS Accelerator config settings.
*/
abstract class GpuOptimisticTransactionBase
(deltaLog: DeltaLog, snapshot: Snapshot, rapidsConf: RapidsConf)
(deltaLog: DeltaLog, snapshot: Snapshot, val rapidsConf: RapidsConf)
(implicit clock: Clock)
extends OptimisticTransaction(deltaLog, snapshot)(clock)
with DeltaLogging {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,12 @@

package com.nvidia.spark.rapids.delta

import java.util.Locale

import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf

import org.apache.spark.network.util.ByteUnit

/** Delta Lake related configs that are not yet provided by Delta Lake. */
trait RapidsDeltaSQLConf {
val OPTIMIZE_WRITE_SMALL_PARTITION_FACTOR =
Expand All @@ -39,6 +43,30 @@ trait RapidsDeltaSQLConf {
.doc("Factor used to rebalance partitions for optimize write.")
.doubleConf
.createWithDefault(1.2)

val AUTO_COMPACT_TARGET =
DeltaSQLConf.buildConf("autoCompact.target")
.internal()
.doc(
"""
|Target files for auto compaction.
| "table", "commit", "partition" options are available. (default: partition)
| If "table", all files in table are eligible for auto compaction.
| If "commit", added/updated files by the commit are eligible.
| If "partition", all files in partitions containing any added/updated files
| by the commit are eligible.
|""".stripMargin
)
.stringConf
.transform(_.toLowerCase(Locale.ROOT))
.createWithDefault("partition")

val AUTO_COMPACT_MAX_COMPACT_BYTES =
DeltaSQLConf.buildConf("autoCompact.maxCompactBytes")
.internal()
.doc("Maximum amount of data for auto compaction.")
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("20GB")
}

object RapidsDeltaSQLConf extends RapidsDeltaSQLConf
Original file line number Diff line number Diff line change
Expand Up @@ -90,16 +90,5 @@ object RapidsDeltaUtils {
}
}
}

val autoCompactEnabled =
getSQLConf("spark.databricks.delta.autoCompact.enabled").orElse {
val metadata = deltaLog.snapshot.metadata
metadata.configuration.get("delta.autoOptimize.autoCompact").orElse {
getSQLConf("spark.databricks.delta.properties.defaults.autoOptimize.autoCompact")
}
}.exists(_.toBoolean)
if (autoCompactEnabled) {
meta.willNotWorkOnGpu("automatic compaction of Delta Lake tables is not supported")
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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 = "GpuDoAutoCompaction"
mythrocks marked this conversation as resolved.
Show resolved Hide resolved

override def run(spark: SparkSession,
txn: OptimisticTransactionImpl,
committedActions: Seq[Action]): Unit = {
val gpuTxn = txn.asInstanceOf[GpuOptimisticTransaction]
val newTxn = new GpuDeltaLog(gpuTxn.deltaLog, gpuTxn.rapidsConf).startTransaction()
new GpuOptimizeExecutor(spark, newTxn, Seq.empty, Seq.empty, committedActions).optimize()
jlowe marked this conversation as resolved.
Show resolved Hide resolved
}

override def handleError(error: Throwable, version: Long): Unit =
throw DeltaErrors.postCommitHookFailedException(this, version, name, error)
}
Original file line number Diff line number Diff line change
Expand Up @@ -252,6 +252,23 @@ class GpuOptimisticTransaction(
identityTracker.foreach { tracker =>
updatedIdentityHighWaterMarks.appendAll(tracker.highWaterMarks.toSeq)
}
resultFiles.toSeq ++ committer.changeFiles
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)
"false" // TODO: Fix getting this from DeltaConfigs.AUTO_COMPACT.
mythrocks marked this conversation as resolved.
Show resolved Hide resolved
}.toBoolean

if (!isOptimize && autoCompactEnabled && fileActions.nonEmpty) {
registerPostCommitHook(GpuDoAutoCompaction)
}

fileActions
}
}
Loading