Skip to content

Commit

Permalink
[Spark][Version Checksum] Incrementally compute VersionChecksum setTr…
Browse files Browse the repository at this point in the history
…ansactions and domainMetadata (#3895)

<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [X] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description

<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->
Follow up for #3828. Adds support
for incrementally computing the set transactions and domain metadata
actions based on the current commit and the last version checksum.
Incremental computation for both these action types have thresholds so
that we don't store them if they are too long (tests have been added for
the same).


## How was this patch tested?

<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->

Added new tests in DomainMetadataSuite and a new suite called
`DeltaIncrementalSetTransactionsSuite`

## Does this PR introduce _any_ user-facing changes?

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
No
  • Loading branch information
dhruvarya-db authored Nov 22, 2024
1 parent 68275d1 commit ec0ab0d
Show file tree
Hide file tree
Showing 7 changed files with 715 additions and 13 deletions.
107 changes: 102 additions & 5 deletions spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala
Original file line number Diff line number Diff line change
Expand Up @@ -78,9 +78,7 @@ trait RecordChecksum extends DeltaLogging {
private lazy val writer =
CheckpointFileManager.create(deltaLog.logPath, deltaLog.newDeltaHadoopConf())

private def getChecksum(snapshot: Snapshot): VersionChecksum = {
snapshot.checksumOpt.getOrElse(snapshot.computeChecksum)
}
private def getChecksum(snapshot: Snapshot): VersionChecksum = snapshot.computeChecksum

protected def writeChecksumFile(txnId: String, snapshot: Snapshot): Unit = {
if (!spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED)) {
Expand Down Expand Up @@ -277,6 +275,12 @@ trait RecordChecksum extends DeltaLogging {
case _ =>
}

val setTransactions = incrementallyComputeSetTransactions(
oldSnapshot, oldVersionChecksum, attemptVersion, actions)

val domainMetadata = incrementallyComputeDomainMetadatas(
oldSnapshot, oldVersionChecksum, attemptVersion, actions)

Right(VersionChecksum(
txnId = txnIdOpt,
tableSizeBytes = tableSizeBytes,
Expand All @@ -286,13 +290,106 @@ trait RecordChecksum extends DeltaLogging {
inCommitTimestampOpt = inCommitTimestamp,
metadata = metadata,
protocol = protocol,
setTransactions = None,
domainMetadata = None,
setTransactions = setTransactions,
domainMetadata = domainMetadata,
histogramOpt = None,
allFiles = None
))
}

/**
* Incrementally compute [[Snapshot.setTransactions]] for the commit `attemptVersion`.
*
* @param oldSnapshot - snapshot corresponding to `attemptVersion` - 1
* @param oldVersionChecksum - [[VersionChecksum]] corresponding to `attemptVersion` - 1
* @param attemptVersion - version which we want to commit
* @param actionsToCommit - actions for commit `attemptVersion`
* @return Optional sequence of incrementally computed [[SetTransaction]]s for commit
* `attemptVersion`.
*/
private def incrementallyComputeSetTransactions(
oldSnapshot: Option[Snapshot],
oldVersionChecksum: VersionChecksum,
attemptVersion: Long,
actionsToCommit: Seq[Action]): Option[Seq[SetTransaction]] = {
// Check-1: check conf
if (!spark.conf.get(DeltaSQLConf.DELTA_WRITE_SET_TRANSACTIONS_IN_CRC)) {
return None
}

// Check-2: check `minSetTransactionRetentionTimestamp` is not set
val newMetadataToCommit = actionsToCommit.collectFirst { case m: Metadata => m }
// TODO: Add support for incrementally computing [[SetTransaction]]s even when
// `minSetTransactionRetentionTimestamp` is set.
// We don't incrementally compute [[SetTransaction]]s when user has configured
// `minSetTransactionRetentionTimestamp` as it makes verification non-deterministic.
// Check all places to figure out whether `minSetTransactionRetentionTimestamp` is set:
// 1. oldSnapshot corresponding to `attemptVersion - 1`
// 2. old VersionChecksum's MetaData (corresponding to `attemptVersion-1`)
// 3. new VersionChecksum's MetaData (corresponding to `attemptVersion`)
val setTransactionRetentionTimestampConfigured =
(oldSnapshot.map(_.metadata) ++ Option(oldVersionChecksum.metadata) ++ newMetadataToCommit)
.exists(DeltaLog.minSetTransactionRetentionInterval(_).nonEmpty)
if (setTransactionRetentionTimestampConfigured) return None

// Check-3: Check old setTransactions are available so that we can incrementally compute new.
val oldSetTransactions = oldVersionChecksum.setTransactions
.getOrElse { return None }

// Check-4: old/new setTransactions are within the threshold.
val setTransactionsToCommit = actionsToCommit.filter(_.isInstanceOf[SetTransaction])
val threshold = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_MAX_SET_TRANSACTIONS_IN_CRC)
if (Math.max(setTransactionsToCommit.size, oldSetTransactions.size) > threshold) return None

// We currently don't attempt incremental [[SetTransaction]] when
// `minSetTransactionRetentionTimestamp` is set. So passing this as None here explicitly.
// We can also ignore file retention because that only affects [[RemoveFile]] actions.
val logReplay = new InMemoryLogReplay(
minFileRetentionTimestamp = 0,
minSetTransactionRetentionTimestamp = None)

logReplay.append(attemptVersion - 1, oldSetTransactions.toIterator)
logReplay.append(attemptVersion, setTransactionsToCommit.toIterator)
Some(logReplay.getTransactions.toSeq).filter(_.size <= threshold)
}

/**
* Incrementally compute [[Snapshot.domainMetadata]] for the commit `attemptVersion`.
*
* @param oldVersionChecksum - [[VersionChecksum]] corresponding to `attemptVersion` - 1
* @param attemptVersion - version which we want to commit
* @param actionsToCommit - actions for commit `attemptVersion`
* @return Sequence of incrementally computed [[DomainMetadata]]s for commit
* `attemptVersion`.
*/
private def incrementallyComputeDomainMetadatas(
oldSnapshot: Option[Snapshot],
oldVersionChecksum: VersionChecksum,
attemptVersion: Long,
actionsToCommit: Seq[Action]): Option[Seq[DomainMetadata]] = {
// Check old DomainMetadatas are available so that we can incrementally compute new.
val oldDomainMetadatas = oldVersionChecksum.domainMetadata
.getOrElse { return None }
val newDomainMetadatas = actionsToCommit.filter(_.isInstanceOf[DomainMetadata])

// We only work with DomainMetadata, so RemoveFile and SetTransaction retention don't matter.
val logReplay = new InMemoryLogReplay(
minFileRetentionTimestamp = 0,
minSetTransactionRetentionTimestamp = None)

val threshold = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_MAX_DOMAIN_METADATAS_IN_CRC)

logReplay.append(attemptVersion - 1, oldDomainMetadatas.iterator)
logReplay.append(attemptVersion, newDomainMetadatas.iterator)
// We don't truncate the set of DomainMetadata actions. Instead, we either store all of them or
// none of them. The advantage of this is that you can then determine presence based on the
// checksum, i.e. if the checksum contains domain metadatas but it doesn't contain the one you
// are looking for, then it's not there.
//
// It's also worth noting that we can distinguish "no domain metadatas" versus
// "domain metadatas not stored" as [[Some]] vs. [[None]].
Some(logReplay.getDomainMetadatas.toSeq).filter(_.size <= threshold)
}
}

object RecordChecksum {
Expand Down
29 changes: 21 additions & 8 deletions spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala
Original file line number Diff line number Diff line change
Expand Up @@ -513,17 +513,29 @@ class Snapshot(
*/
def computeChecksum: VersionChecksum = VersionChecksum(
txnId = None,
tableSizeBytes = sizeInBytes,
numFiles = numOfFiles,
numMetadata = numOfMetadata,
numProtocol = numOfProtocol,
inCommitTimestampOpt = getInCommitTimestampOpt,
setTransactions = checksumOpt.flatMap(_.setTransactions),
domainMetadata = checksumOpt.flatMap(_.domainMetadata),
metadata = metadata,
protocol = protocol,
histogramOpt = checksumOpt.flatMap(_.histogramOpt),
allFiles = checksumOpt.flatMap(_.allFiles))
allFiles = checksumOpt.flatMap(_.allFiles),
tableSizeBytes = checksumOpt.map(_.tableSizeBytes).getOrElse(sizeInBytes),
numFiles = checksumOpt.map(_.numFiles).getOrElse(numOfFiles),
numMetadata = checksumOpt.map(_.numMetadata).getOrElse(numOfMetadata),
numProtocol = checksumOpt.map(_.numProtocol).getOrElse(numOfProtocol),
// Only return setTransactions and domainMetadata if they are either already present
// in the checksum or if they have already been computed in the current snapshot.
setTransactions = checksumOpt.flatMap(_.setTransactions)
.orElse {
Option.when(_computedStateTriggered &&
// Only extract it from the current snapshot if set transaction
// writes are enabled.
spark.conf.get(DeltaSQLConf.DELTA_WRITE_SET_TRANSACTIONS_IN_CRC)) {
setTransactions
}
},
domainMetadata = checksumOpt.flatMap(_.domainMetadata)
.orElse(Option.when(_computedStateTriggered)(domainMetadata)),
histogramOpt = checksumOpt.flatMap(_.histogramOpt)
)

/** Returns the data schema of the table, used for reading stats */
def tableSchema: StructType = metadata.dataSchema
Expand Down Expand Up @@ -704,6 +716,7 @@ class DummySnapshot(

override protected lazy val computedState: SnapshotState = initialState(metadata, protocol)
override protected lazy val getInCommitTimestampOpt: Option[Long] = None
_computedStateTriggered = true

// The [[InitialSnapshot]] is not backed by any external commit-coordinator.
override val tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient] = None
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1271,6 +1271,7 @@ trait SnapshotManagement { self: DeltaLog =>
// a checksum based on state reconstruction. Disable incremental commit to avoid
// further error triggers in this session.
spark.sessionState.conf.setConf(DeltaSQLConf.INCREMENTAL_COMMIT_ENABLED, false)
spark.sessionState.conf.setConf(DeltaSQLConf.DELTA_WRITE_SET_TRANSACTIONS_IN_CRC, false)
return createSnapshotAfterCommit(
initSegment,
newChecksumOpt = None,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,9 @@ trait SnapshotStateManager extends DeltaLogging { self: Snapshot =>

// For implicits which re-use Encoder:
import implicits._
/** Whether computedState is already computed or not */
@volatile protected var _computedStateTriggered: Boolean = false


/** A map to look up transaction version by appId. */
lazy val transactions: Map[String, Long] = setTransactions.map(t => t.appId -> t.version).toMap
Expand Down Expand Up @@ -114,6 +117,7 @@ trait SnapshotStateManager extends DeltaLogging { self: Snapshot =>
throw DeltaErrors.actionNotFoundException("metadata", version)
}

_computedStateTriggered = true
_computedState
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1073,6 +1073,31 @@ trait DeltaSQLConfBase {
.booleanConf
.createWithDefault(true)

val DELTA_WRITE_SET_TRANSACTIONS_IN_CRC =
buildConf("setTransactionsInCrc.writeOnCommit")
.internal()
.doc("When enabled, each commit will incrementally compute and cache all SetTransaction" +
" actions in the .crc file. Note that this only happens when incremental commits" +
s" are enabled (${INCREMENTAL_COMMIT_ENABLED.key})")
.booleanConf
.createWithDefault(true)

val DELTA_MAX_SET_TRANSACTIONS_IN_CRC =
buildConf("setTransactionsInCrc.maxAllowed")
.internal()
.doc("Threshold of the number of SetTransaction actions below which this optimization" +
" should be enabled")
.longConf
.createWithDefault(100)

val DELTA_MAX_DOMAIN_METADATAS_IN_CRC =
buildConf("domainMetadatasInCrc.maxAllowed")
.internal()
.doc("Threshold of the number of DomainMetadata actions below which this optimization" +
" should be enabled")
.longConf
.createWithDefault(10)

val DELTA_CHECKPOINT_THROW_EXCEPTION_WHEN_FAILED =
buildConf("checkpoint.exceptionThrowing.enabled")
.internal()
Expand Down
Loading

0 comments on commit ec0ab0d

Please sign in to comment.