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

[Spark] Pass table identifier from DeltaLog constructor to commit coordinator #3718

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all 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 @@ -76,6 +76,7 @@ class DeltaLog private(
val dataPath: Path,
val options: Map[String, String],
val allOptions: Map[String, String],
val initialTableIdentifier: Option[TableIdentifier],
val clock: Clock
) extends Checkpoints
with MetadataCleanup
Expand Down Expand Up @@ -862,6 +863,7 @@ object DeltaLog extends DeltaLogging {
dataPath = path.getParent,
options = fileSystemOptions,
allOptions = options,
initialTableIdentifier = None,
clock = clock
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path}
import org.apache.spark.{SparkContext, SparkException}
import org.apache.spark.internal.MDC
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.util.{ThreadUtils, Utils}

/**
Expand Down Expand Up @@ -155,6 +156,8 @@ trait SnapshotManagement { self: DeltaLog =>
* @param startVersion the version to start. Inclusive.
* @param tableCommitCoordinatorClientOpt the optional commit coordinator to use for fetching
* un-backfilled commits.
* @param tableIdentifierOpt the optional table identifier to pass to the commit coordinator
* client.
* @param versionToLoad the optional parameter to set the max version we should return. Inclusive.
* @param includeMinorCompactions Whether to include minor compaction files in the result
* @return A tuple where the first element is an array of log files (possibly empty, if no
Expand All @@ -164,6 +167,7 @@ trait SnapshotManagement { self: DeltaLog =>
protected def listDeltaCompactedDeltaCheckpointFilesAndLatestChecksumFile(
startVersion: Long,
tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
tableIdentifierOpt: Option[TableIdentifier],
versionToLoad: Option[Long],
includeMinorCompactions: Boolean): (Option[Array[FileStatus]], Option[FileStatus]) = {
val tableCommitCoordinatorClient = tableCommitCoordinatorClientOpt.getOrElse {
Expand All @@ -174,9 +178,6 @@ trait SnapshotManagement { self: DeltaLog =>

// Submit a potential async call to get commits from commit coordinator if available
val threadPool = SnapshotManagement.commitCoordinatorGetCommitsThreadPool
// TODO(table-identifier-plumbing): Plumb the right tableIdentifier from the deltaLog.update and
// Cold deltaLog initialization codepath.
val tableIdentifierOpt = None
def getCommitsTask(isAsyncRequest: Boolean): GetCommitsResponse = {
CoordinatedCommitsUtils.getCommitsFromCommitCoordinatorWithUsageLogs(
this, tableCommitCoordinatorClient, tableIdentifierOpt,
Expand Down Expand Up @@ -322,12 +323,18 @@ trait SnapshotManagement { self: DeltaLog =>
protected final def listDeltaCompactedDeltaAndCheckpointFiles(
startVersion: Long,
tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
tableIdentifierOpt: Option[TableIdentifier],
versionToLoad: Option[Long],
includeMinorCompactions: Boolean): Option[Array[FileStatus]] = {
recordDeltaOperation(self, "delta.deltaLog.listDeltaAndCheckpointFiles") {
val (logTuplesOpt, latestChecksumOpt) =
listDeltaCompactedDeltaCheckpointFilesAndLatestChecksumFile(
startVersion, tableCommitCoordinatorClientOpt, versionToLoad, includeMinorCompactions)
startVersion,
tableCommitCoordinatorClientOpt,
tableIdentifierOpt,
versionToLoad,
includeMinorCompactions
)
lastSeenChecksumFileStatusOpt = latestChecksumOpt
logTuplesOpt
}
Expand All @@ -354,6 +361,7 @@ trait SnapshotManagement { self: DeltaLog =>
versionToLoad: Option[Long] = None,
oldCheckpointProviderOpt: Option[UninitializedCheckpointProvider] = None,
tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient] = None,
tableIdentifierOpt: Option[TableIdentifier] = None,
lastCheckpointInfo: Option[LastCheckpointInfo] = None): Option[LogSegment] = {
// List based on the last known checkpoint version.
// if that is -1, list from version 0L
Expand All @@ -362,21 +370,31 @@ trait SnapshotManagement { self: DeltaLog =>
val includeMinorCompactions =
spark.conf.get(DeltaSQLConf.DELTALOG_MINOR_COMPACTION_USE_FOR_READS)
val newFiles = listDeltaCompactedDeltaAndCheckpointFiles(
listingStartVersion, tableCommitCoordinatorClientOpt, versionToLoad, includeMinorCompactions)
listingStartVersion,
tableCommitCoordinatorClientOpt,
tableIdentifierOpt,
versionToLoad,
includeMinorCompactions
)
getLogSegmentForVersion(
versionToLoad,
newFiles,
validateLogSegmentWithoutCompactedDeltas = true,
oldCheckpointProviderOpt = oldCheckpointProviderOpt,
tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
tableIdentifierOpt = tableIdentifierOpt,
lastCheckpointInfo = lastCheckpointInfo
)
}

private def createLogSegment(previousSnapshot: Snapshot): Option[LogSegment] = {
private def createLogSegment(
previousSnapshot: Snapshot,
tableIdentifierOpt: Option[TableIdentifier]): Option[LogSegment] = {
createLogSegment(
oldCheckpointProviderOpt = Some(previousSnapshot.checkpointProvider),
tableCommitCoordinatorClientOpt = previousSnapshot.tableCommitCoordinatorClientOpt)
tableCommitCoordinatorClientOpt = previousSnapshot.tableCommitCoordinatorClientOpt,
tableIdentifierOpt = tableIdentifierOpt
)
}

/**
Expand Down Expand Up @@ -439,6 +457,7 @@ trait SnapshotManagement { self: DeltaLog =>
files: Option[Array[FileStatus]],
validateLogSegmentWithoutCompactedDeltas: Boolean,
tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
tableIdentifierOpt: Option[TableIdentifier] = None,
oldCheckpointProviderOpt: Option[UninitializedCheckpointProvider],
lastCheckpointInfo: Option[LastCheckpointInfo]): Option[LogSegment] = {
recordFrameProfile("Delta", "SnapshotManagement.getLogSegmentForVersion") {
Expand Down Expand Up @@ -483,8 +502,13 @@ trait SnapshotManagement { self: DeltaLog =>
recordDeltaEvent(this, "delta.checkpoint.error.partial")
val snapshotVersion = versionToLoad.getOrElse(deltaVersion(deltas.last))
getLogSegmentWithMaxExclusiveCheckpointVersion(
snapshotVersion, lastCheckpointVersion, tableCommitCoordinatorClientOpt)
.foreach { alternativeLogSegment => return Some(alternativeLogSegment) }
snapshotVersion,
lastCheckpointVersion,
tableCommitCoordinatorClientOpt,
tableIdentifierOpt
).foreach { alternativeLogSegment =>
return Some(alternativeLogSegment)
}

// No alternative found, but the directory contains files so we cannot return None.
throw DeltaErrors.missingPartFilesException(
Expand Down Expand Up @@ -638,11 +662,13 @@ trait SnapshotManagement { self: DeltaLog =>
val lastCheckpointOpt = readLastCheckpointFile()
val initialSegmentForNewSnapshot = createLogSegment(
versionToLoad = None,
tableIdentifierOpt = initialTableIdentifier,
lastCheckpointInfo = lastCheckpointOpt)
val snapshot = getUpdatedSnapshot(
oldSnapshotOpt = None,
initialSegmentForNewSnapshot = initialSegmentForNewSnapshot,
initialTableCommitCoordinatorClient = None,
tableIdentifierOpt = initialTableIdentifier,
isAsync = false)
CapturedSnapshot(snapshot, snapshotInitWallclockTime)
}
Expand Down Expand Up @@ -712,7 +738,8 @@ trait SnapshotManagement { self: DeltaLog =>
private def getLogSegmentWithMaxExclusiveCheckpointVersion(
snapshotVersion: Long,
maxExclusiveCheckpointVersion: Long,
tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient]): Option[LogSegment] = {
tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
tableIdentifierOpt: Option[TableIdentifier] = None): Option[LogSegment] = {
assert(
snapshotVersion >= maxExclusiveCheckpointVersion,
s"snapshotVersion($snapshotVersion) is less than " +
Expand All @@ -725,6 +752,7 @@ trait SnapshotManagement { self: DeltaLog =>
val filesSinceCheckpointVersion = listDeltaCompactedDeltaAndCheckpointFiles(
startVersion = cp.version,
tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
tableIdentifierOpt = tableIdentifierOpt,
versionToLoad = Some(snapshotVersion),
includeMinorCompactions = false
).getOrElse(Array.empty)
Expand Down Expand Up @@ -767,6 +795,7 @@ trait SnapshotManagement { self: DeltaLog =>
listDeltaCompactedDeltaAndCheckpointFiles(
startVersion = 0,
tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
tableIdentifierOpt = tableIdentifierOpt,
versionToLoad = Some(snapshotVersion),
includeMinorCompactions = false)
val (deltas, deltaVersions) =
Expand Down Expand Up @@ -904,6 +933,7 @@ trait SnapshotManagement { self: DeltaLog =>
val newFilesOpt = listDeltaCompactedDeltaAndCheckpointFiles(
startVersion = oldLogSegment.version + 1,
tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
tableIdentifierOpt = None,
versionToLoad = None,
includeMinorCompactions = includeCompactions)
val newFiles = newFilesOpt.getOrElse {
Expand Down Expand Up @@ -1049,7 +1079,7 @@ trait SnapshotManagement { self: DeltaLog =>
recordDeltaOperation(this, "delta.log.update", Map(TAG_ASYNC -> isAsync.toString)) {
val updateStartTimeMs = clock.getTimeMillis()
val previousSnapshot = currentSnapshot.snapshot
val segmentOpt = createLogSegment(previousSnapshot)
val segmentOpt = createLogSegment(previousSnapshot, tableIdentifierOpt = None)
val newSnapshot = getUpdatedSnapshot(
oldSnapshotOpt = Some(previousSnapshot),
initialSegmentForNewSnapshot = segmentOpt,
Expand All @@ -1073,7 +1103,8 @@ trait SnapshotManagement { self: DeltaLog =>
oldSnapshotOpt: Option[Snapshot],
initialSegmentForNewSnapshot: Option[LogSegment],
initialTableCommitCoordinatorClient: Option[TableCommitCoordinatorClient],
isAsync: Boolean): Snapshot = {
isAsync: Boolean,
tableIdentifierOpt: Option[TableIdentifier] = None): Snapshot = {
var newSnapshot = getSnapshotForLogSegmentInternal(
oldSnapshotOpt,
initialSegmentForNewSnapshot,
Expand All @@ -1089,7 +1120,7 @@ trait SnapshotManagement { self: DeltaLog =>
initialTableCommitCoordinatorClient.forall(!_.semanticsEquals(newStore))
}
if (usedStaleCommitCoordinator) {
val segmentOpt = createLogSegment(newSnapshot)
val segmentOpt = createLogSegment(newSnapshot, tableIdentifierOpt)
newSnapshot =
getSnapshotForLogSegmentInternal(
Some(newSnapshot), segmentOpt, newSnapshot.tableCommitCoordinatorClientOpt, isAsync)
Expand Down
Loading