Skip to content

Commit

Permalink
add retry logic for delta uniform iceberg conversion (#3856)
Browse files Browse the repository at this point in the history
<!--
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.
-->

add retry logic for delta uniform iceberg conversion

## 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.
-->

## 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'.
-->
  • Loading branch information
lzlfred authored Nov 9, 2024
1 parent 95d493c commit e65d06e
Show file tree
Hide file tree
Showing 3 changed files with 77 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.delta.icebergShaded.IcebergTransactionUtils._
import org.apache.spark.sql.delta.logging.DeltaLogKeys
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.schema.SchemaUtils
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.hadoop.conf.Configuration
import shadedForDelta.org.apache.iceberg.{AppendFiles, DeleteFiles, OverwriteFiles, PendingUpdate, RewriteFiles, Transaction => IcebergTransaction}
Expand All @@ -37,6 +38,7 @@ import shadedForDelta.org.apache.iceberg.mapping.MappingUtil
import shadedForDelta.org.apache.iceberg.mapping.NameMappingParser

import org.apache.spark.internal.MDC
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.catalog.CatalogTable

sealed trait IcebergTableOp
Expand All @@ -56,6 +58,7 @@ case object REPLACE_TABLE extends IcebergTableOp
* @param lastConvertedDeltaVersion the delta version this Iceberg txn starts from.
*/
class IcebergConversionTransaction(
protected val spark: SparkSession,
protected val catalogTable: CatalogTable,
protected val conf: Configuration,
protected val postCommitSnapshot: Snapshot,
Expand Down Expand Up @@ -342,11 +345,18 @@ class IcebergConversionTransaction(
// possible legitimate Delta version which is 0.
val deltaVersion = if (tableOp == CREATE_TABLE) -1 else postCommitSnapshot.version

txn.updateProperties()
.set(IcebergConverter.DELTA_VERSION_PROPERTY, deltaVersion.toString)
var updateTxn = txn.updateProperties()
updateTxn = updateTxn.set(IcebergConverter.DELTA_VERSION_PROPERTY, deltaVersion.toString)
.set(IcebergConverter.DELTA_TIMESTAMP_PROPERTY, postCommitSnapshot.timestamp.toString)
.set(IcebergConstants.ICEBERG_NAME_MAPPING_PROPERTY, nameMapping)
.commit()

if (spark.sessionState.conf.getConf(
DeltaSQLConf.DELTA_UNIFORM_ICEBERG_INCLUDE_BASE_CONVERTED_VERSION)) {
lastConvertedDeltaVersion.foreach { v =>
updateTxn = updateTxn.set(IcebergConverter.BASE_DELTA_VERSION_PROPERTY, v.toString)
}
}
updateTxn.commit()

// We ensure the iceberg txns are serializable by only allowing them to commit against
// lastConvertedIcebergSnapshotId.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.hadoop.fs.Path
import shadedForDelta.org.apache.iceberg.{Table => IcebergTable}
import shadedForDelta.org.apache.iceberg.exceptions.CommitFailedException
import shadedForDelta.org.apache.iceberg.hive.{HiveCatalog, HiveTableOperations}

import org.apache.spark.internal.MDC
Expand All @@ -54,8 +55,17 @@ object IcebergConverter {
*/
val DELTA_TIMESTAMP_PROPERTY = "delta-timestamp"

/**
* Property to be set in translated Iceberg metadata files.
* Indicates the base delta commit version # that the conversion started from
*/
val BASE_DELTA_VERSION_PROPERTY = "base-delta-version"

def getLastConvertedDeltaVersion(table: Option[IcebergTable]): Option[Long] =
table.flatMap(_.properties().asScala.get(DELTA_VERSION_PROPERTY)).map(_.toLong)

def getLastConvertedDeltaTimestamp(table: Option[IcebergTable]): Option[Long] =
table.flatMap(_.properties().asScala.get(DELTA_TIMESTAMP_PROPERTY)).map(_.toLong)
}

/**
Expand Down Expand Up @@ -177,7 +187,7 @@ class IcebergConverter(spark: SparkSession)
override def convertSnapshot(
snapshotToConvert: Snapshot, catalogTable: CatalogTable): Option[(Long, Long)] = {
try {
convertSnapshot(snapshotToConvert, None, catalogTable)
convertSnapshotWithRetry(snapshotToConvert, None, catalogTable)
} catch {
case NonFatal(e) =>
logError(log"Error when converting to Iceberg metadata", e)
Expand Down Expand Up @@ -205,7 +215,7 @@ class IcebergConverter(spark: SparkSession)
snapshotToConvert: Snapshot, txn: OptimisticTransactionImpl): Option[(Long, Long)] = {
try {
txn.catalogTable match {
case Some(table) => convertSnapshot(snapshotToConvert, Some(txn), table)
case Some(table) => convertSnapshotWithRetry(snapshotToConvert, Some(txn), table)
case _ =>
val msg = s"CatalogTable for table ${snapshotToConvert.deltaLog.tableId} " +
s"is empty in txn. Skip iceberg conversion."
Expand All @@ -227,6 +237,41 @@ class IcebergConverter(spark: SparkSession)
}
}

/**
* Convert the specified snapshot into Iceberg with retry
*/
private def convertSnapshotWithRetry(
snapshotToConvert: Snapshot,
txnOpt: Option[OptimisticTransactionImpl],
catalogTable: CatalogTable,
maxRetry: Int =
spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_UNIFORM_ICEBERG_RETRY_TIMES)
): Option[(Long, Long)] = {
var retryAttempt = 0
while (retryAttempt < maxRetry) {
try {
return convertSnapshot(snapshotToConvert, txnOpt, catalogTable)
} catch {
case e: CommitFailedException if retryAttempt < maxRetry =>
retryAttempt += 1
val lastConvertedIcebergTable = loadIcebergTable(snapshotToConvert, catalogTable)
val lastDeltaVersionConverted = IcebergConverter
.getLastConvertedDeltaVersion(lastConvertedIcebergTable)
val lastConvertedDeltaTimestamp = IcebergConverter
.getLastConvertedDeltaTimestamp(lastConvertedIcebergTable)
// Do not retry if the current or higher Delta version is already converted
(lastDeltaVersionConverted, lastConvertedDeltaTimestamp) match {
case (Some(version), Some(timestamp)) if version >= snapshotToConvert.version =>
return Some(version, timestamp)
case _ =>
logWarning(s"CommitFailedException when converting to Iceberg metadata;" +
s" retry count $retryAttempt", e)
}
}
}
throw new IllegalStateException("should not happen")
}

/**
* Convert the specified snapshot into Iceberg. NOTE: This operation is blocking. Call
* enqueueSnapshotForConversion to run the operation asynchronously.
Expand Down Expand Up @@ -288,7 +333,7 @@ class IcebergConverter(spark: SparkSession)
}

val icebergTxn = new IcebergConversionTransaction(
cleanedCatalogTable, log.newDeltaHadoopConf(), snapshotToConvert, tableOp,
spark, cleanedCatalogTable, log.newDeltaHadoopConf(), snapshotToConvert, tableOp,
lastConvertedIcebergSnapshotId, lastDeltaVersionConverted)

// Write out the actions taken since the last conversion (or since table creation).
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1412,6 +1412,22 @@ trait DeltaSQLConfBase {
.booleanConf
.createWithDefault(false)

val DELTA_UNIFORM_ICEBERG_RETRY_TIMES =
buildConf("uniform.iceberg.retry.times")
.doc("The number of retries iceberg conversions should have in case " +
"of failures")
.internal()
.intConf
.createWithDefault(3)

val DELTA_UNIFORM_ICEBERG_INCLUDE_BASE_CONVERTED_VERSION =
buildConf("uniform.iceberg.include.base.converted.version")
.doc("If true, include the base converted delta version as a tbl property in Iceberg " +
"metadata to indicate the delta version that the conversion started from")
.internal()
.booleanConf
.createWithDefault(true)

val DELTA_OPTIMIZE_MIN_FILE_SIZE =
buildConf("optimize.minFileSize")
.internal()
Expand Down

0 comments on commit e65d06e

Please # to comment.