Skip to content

Commit

Permalink
Refactor for Delta Uniform IcebergConversionTransaction to use curren…
Browse files Browse the repository at this point in the history
…t iceberg PartitionSpec (#4174)

<!--
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

Refactor for Delta Uniform IcebergConversionTransaction to use current
iceberg PartitionSpec.

## How was this patch tested?

UT

## 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 Feb 20, 2025
1 parent 3e3c01b commit 36d1dae
Showing 1 changed file with 15 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ 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.{AppendFiles, DeleteFiles, OverwriteFiles, PartitionSpec, PendingUpdate, RewriteFiles, Transaction => IcebergTransaction}
import shadedForDelta.org.apache.iceberg.ExpireSnapshots
import shadedForDelta.org.apache.iceberg.mapping.MappingUtil
import shadedForDelta.org.apache.iceberg.mapping.NameMappingParser
Expand Down Expand Up @@ -64,7 +64,8 @@ class IcebergConversionTransaction(
protected val postCommitSnapshot: Snapshot,
protected val tableOp: IcebergTableOp = WRITE_TABLE,
protected val lastConvertedIcebergSnapshotId: Option[Long] = None,
protected val lastConvertedDeltaVersion: Option[Long] = None) extends DeltaLogging {
protected val lastConvertedDeltaVersion: Option[Long] = None
) extends DeltaLogging {

///////////////////////////
// Nested Helper Classes //
Expand Down Expand Up @@ -100,7 +101,7 @@ class IcebergConversionTransaction(
convertDeltaAddFileToIcebergDataFile(
add,
tablePath,
partitionSpec,
currentPartitionSpec,
logicalToPhysicalPartitionNames,
statsParser,
postCommitSnapshot
Expand Down Expand Up @@ -141,7 +142,7 @@ class IcebergConversionTransaction(
convertDeltaAddFileToIcebergDataFile(
add,
tablePath,
partitionSpec,
currentPartitionSpec,
logicalToPhysicalPartitionNames,
statsParser,
postCommitSnapshot
Expand All @@ -154,7 +155,7 @@ class IcebergConversionTransaction(
convertDeltaRemoveFileToIcebergDataFile(
remove,
tablePath,
partitionSpec,
currentPartitionSpec,
logicalToPhysicalPartitionNames,
postCommitSnapshot)
)
Expand All @@ -178,7 +179,7 @@ class IcebergConversionTransaction(
convertDeltaRemoveFileToIcebergDataFile(
f,
tablePath,
partitionSpec,
currentPartitionSpec,
logicalToPhysicalPartitionNames,
postCommitSnapshot)
}.toSet.asJava
Expand All @@ -188,7 +189,7 @@ class IcebergConversionTransaction(
convertDeltaAddFileToIcebergDataFile(
f,
tablePath,
partitionSpec,
currentPartitionSpec,
logicalToPhysicalPartitionNames,
statsParser,
postCommitSnapshot
Expand All @@ -212,8 +213,15 @@ class IcebergConversionTransaction(
protected val tablePath = postCommitSnapshot.deltaLog.dataPath
protected val icebergSchema =
convertDeltaSchemaToIcebergSchema(postCommitSnapshot.metadata.schema)
// Initial partition spec converted from Delta
protected val partitionSpec =
createPartitionSpec(icebergSchema, postCommitSnapshot.metadata.partitionColumns)

// Current partition spec from iceberg table
def currentPartitionSpec: PartitionSpec = {
Some(txn.table()).map(_.spec()).getOrElse(partitionSpec)
}

private val logicalToPhysicalPartitionNames =
getPartitionPhysicalNameMapping(postCommitSnapshot.metadata.partitionSchema)

Expand Down

0 comments on commit 36d1dae

Please sign in to comment.