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

Uniform: Remove snapshot expiration patch, replace with custom delete callback which checks against metadata location #4059

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 @@ -32,10 +32,10 @@ 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}
import shadedForDelta.org.apache.iceberg.ExpireSnapshots
import shadedForDelta.org.apache.iceberg.{AppendFiles, DeleteFiles, ExpireSnapshots, OverwriteFiles, PendingUpdate, RewriteFiles, TableProperties, Transaction => IcebergTransaction}
import shadedForDelta.org.apache.iceberg.mapping.MappingUtil
import shadedForDelta.org.apache.iceberg.mapping.NameMappingParser
import shadedForDelta.org.apache.iceberg.util.LocationUtil

import org.apache.spark.internal.MDC
import org.apache.spark.sql.SparkSession
Expand Down Expand Up @@ -262,7 +262,26 @@ class IcebergConversionTransaction(
}

def getExpireSnapshotHelper(): ExpireSnapshotHelper = {
val ret = new ExpireSnapshotHelper(txn.expireSnapshots())
val table = txn.table()
val tableLocation = LocationUtil.stripTrailingSlash(table.location)
val defaultWriteMetadataLocation = s"$tableLocation/metadata"
val writeMetadataLocation = LocationUtil.stripTrailingSlash(
table.properties().getOrDefault(
TableProperties.WRITE_METADATA_LOCATION, defaultWriteMetadataLocation))
val expireSnapshots = if (tablePath.toString == writeMetadataLocation) {
// Don't attempt any file cleanup in the edge-case configuration
// that the data location (in Uniform the table root location)
// is the same as the Iceberg metadata location
txn.expireSnapshots().cleanExpiredFiles(false)
} else {
txn.expireSnapshots().deleteWith(path => {
if (path.startsWith(writeMetadataLocation)) {
table.io().deleteFile(path)
}
})
}

val ret = new ExpireSnapshotHelper(expireSnapshots)
fileUpdates += ret
ret
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,14 @@ import org.scalatest.time.SpanSugar._
import org.apache.spark.SparkContext
import org.apache.spark.sql.{QueryTest, Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogStorageFormat}
import org.apache.spark.sql.delta.actions.Metadata
import org.apache.spark.sql.types.{IntegerType, StringType, StructType, StructField}
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
import org.apache.spark.sql.delta.actions.{Metadata, RemoveFile}
import org.apache.spark.sql.delta.implicits._
import org.apache.spark.sql.delta.icebergShaded.IcebergTransactionUtils
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
import org.apache.spark.util.Utils
import shadedForDelta.org.apache.iceberg.{Table, TableProperties}

/**
* This test suite relies on an external Hive metastore (HMS) instance to run.
Expand Down Expand Up @@ -110,7 +114,8 @@ class ConvertToIcebergSuite extends QueryTest with Eventually {
s"""CREATE TABLE `${testTableName}` (col1 INT) USING DELTA
|TBLPROPERTIES (
| 'delta.columnMapping.mode' = 'name',
| 'delta.universalFormat.enabledFormats' = 'iceberg'
| 'delta.universalFormat.enabledFormats' = 'iceberg',
| 'delta.enableIcebergCompatV2' = 'true'
|)""".stripMargin)
runDeltaSql(s"INSERT INTO `$testTableName` VALUES (123)")
verifyReadWithIceberg(testTableName, Seq(Row(123)))
Expand All @@ -123,21 +128,160 @@ class ConvertToIcebergSuite extends QueryTest with Eventually {
withDefaultTablePropsInSQLConf {
deltaSpark.range(10).write.format("delta")
.option("path", testTablePath)
.option("delta.enableIcebergCompatV2", "true")
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

None of these tests passed for me (even before my change) without setting this so I'm fixing it in this PR

.saveAsTable(testTableName)
}
}
withDeltaSparkSession { deltaSpark =>
deltaSpark.range(10, 20, 1)
.write.format("delta").mode("append")
.option("path", testTablePath)
.option("delta.enableIcebergCompatV2", "true")
.saveAsTable(testTableName)
}
verifyReadWithIceberg(testTableName, 0 to 19 map (Row(_)))
}
}

def runDeltaSql(sqlStr: String): Unit = {
test("Expire Snapshots") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

put this case in spark/src/test/scala/org/apache/spark/sql/delta/UniversalFormatSuiteBase.scala

if (hmsReady(PORT)) {
runDeltaSql(
s"""CREATE TABLE `${testTableName}` (col1 INT) USING DELTA
|TBLPROPERTIES (
| 'delta.columnMapping.mode' = 'name',
| 'delta.universalFormat.enabledFormats' = 'iceberg',
| 'delta.enableIcebergCompatV2' = 'true'
|
|)""".stripMargin)

val icebergTable = loadIcebergTable()
icebergTable.updateProperties().set(TableProperties.MAX_SNAPSHOT_AGE_MS, "1").commit()

for (i <- 0 to 7) {
runDeltaSql(s"INSERT INTO ${testTableName} VALUES (${i})",
DeltaSQLConf.DELTA_UNIFORM_ICEBERG_SYNC_CONVERT_ENABLED.key -> "true")
}

// Sleep past snapshot retention duration
Thread.sleep(5)
withIcebergSparkSession { icebergSpark => {
icebergSpark.sql(s"REFRESH TABLE $testTableName")
val manifestListsBeforeExpiration = manifestListLocations(icebergSpark)
assert(manifestListsBeforeExpiration.length == 8)
val manifestsBeforeExpiration = manifestLocations(icebergSpark)

// Trigger snapshot expiration
runDeltaSql(s"OPTIMIZE ${testTableName}")
icebergSpark.sql(s"REFRESH TABLE $testTableName")

// Manifest lists from earlier snapshots should be cleaned up
val manifestListsAfterExpiration = manifestListLocations(icebergSpark)
assert(manifestListsAfterExpiration.length == 1)
assertAllFilesDeleted(icebergTable, manifestListsBeforeExpiration)

// Unreachable manifests should be cleaned up
val manifestsAfterExpiration = manifestLocations(icebergSpark)

val unreachableManifests = manifestsBeforeExpiration.diff(manifestsAfterExpiration)
assertAllFilesDeleted(icebergTable, unreachableManifests)
}}

withDeltaSparkSession(deltaSparkSession => {
val deltaLog = DeltaLog.forTable(deltaSparkSession, testTablePath)
val logicallyRemovedDataFiles = deltaLog.getChanges(9).toArray.head._2.collect {
case removeFile: RemoveFile => removeFile.absolutePath(deltaLog)
}

// The data files must not be cleaned up
withIcebergSparkSession(_ => {
val table = loadIcebergTable()
logicallyRemovedDataFiles.foreach(
file => assert(table.io().newInputFile(file.toString).exists()))
})
})
}
}

test("Expire Snapshots doesn't cleanup in case the data/metadata locations are the same") {
if (hmsReady(PORT)) {
runDeltaSql(
s"""CREATE TABLE `${testTableName}` (col1 INT) USING DELTA
|TBLPROPERTIES (
| 'delta.columnMapping.mode' = 'name',
| 'delta.universalFormat.enabledFormats' = 'iceberg',
| 'delta.enableIcebergCompatV2' = 'true'
|
|)""".stripMargin)

val icebergTable = loadIcebergTable()
icebergTable.updateProperties()
.set(TableProperties.MAX_SNAPSHOT_AGE_MS, "1")
.set(TableProperties.WRITE_METADATA_LOCATION, icebergTable.location())
.commit()

for (i <- 0 to 7) {
runDeltaSql(s"INSERT INTO ${testTableName} VALUES (${i})",
DeltaSQLConf.DELTA_UNIFORM_ICEBERG_SYNC_CONVERT_ENABLED.key -> "true")
}

// Sleep past snapshot retention duration
Thread.sleep(5)

withIcebergSparkSession { icebergSpark => {
icebergSpark.sql(s"REFRESH TABLE $testTableName")
val manifestListsBeforeExpiration = manifestListLocations(icebergSpark)
assert(manifestListsBeforeExpiration.length == 8)

// Trigger snapshot expiration
runDeltaSql(s"OPTIMIZE ${testTableName}")
icebergSpark.sql(s"REFRESH TABLE $testTableName")

// Manifest lists from earlier snapshots should not be cleaned up
val manifestListsAfterExpiration = manifestListLocations(icebergSpark)
manifestListsAfterExpiration.foreach(
file => assert(icebergTable.io().newInputFile(file).exists()))
}
}
}
}

private def manifestListLocations(icebergSparkSession: SparkSession): Array[String] = {
icebergSparkSession
.sql(s"SELECT * FROM default.${testTableName}.snapshots")
.select("manifest_list")
.as[String]
.collect()
}

private def manifestLocations(icebergSparkSession: SparkSession): Array[String] = {
icebergSparkSession
.sql(s"SELECT * FROM default.${testTableName}.manifests")
.select("path")
.as[String]
.collect()
}

private def loadIcebergTable(): shadedForDelta.org.apache.iceberg.Table = {
withDeltaSparkSession { deltaSpark => {
val log = DeltaLog.forTable(deltaSpark, testTablePath)
val hiveCatalog = IcebergTransactionUtils.createHiveCatalog(
log.newDeltaHadoopConf()
)
val table = hiveCatalog.loadTable(
shadedForDelta.org.apache.iceberg.catalog.TableIdentifier
.of("default", testTableName)
)
table
}}
}

private def assertAllFilesDeleted(icebergTable: Table, files: Array[String]): Unit = {
files.foreach(file => assert(!icebergTable.io().newInputFile(file).exists()))
}

def runDeltaSql(sqlStr: String, conf: (String, String)*): Unit = {
withDeltaSparkSession { deltaSpark =>
conf.foreach(c => deltaSpark.conf.set(c._1, c._2))
deltaSpark.sql(sqlStr)
}
}
Expand Down
Loading
Loading