@@ -23,16 +23,17 @@ import scala.collection.JavaConverters._
2323import scala .util .control .Breaks ._
2424import scala .util .control .NonFatal
2525
26- import org .apache .spark .sql .delta .{DeltaFileNotFoundException , DeltaFileProviderUtils , OptimisticTransactionImpl , Snapshot , UniversalFormatConverter }
26+ import org .apache .spark .sql .delta .{DeltaFileNotFoundException , DeltaFileProviderUtils , OptimisticTransactionImpl , Snapshot , UniversalFormat , UniversalFormatConverter }
2727import org .apache .spark .sql .delta .actions .{Action , AddFile , CommitInfo , RemoveFile }
2828import org .apache .spark .sql .delta .hooks .IcebergConverterHook
2929import org .apache .spark .sql .delta .metering .DeltaLogging
3030import org .apache .spark .sql .delta .sources .DeltaSQLConf
3131import org .apache .commons .lang3 .exception .ExceptionUtils
3232import org .apache .hadoop .fs .Path
33- import shadedForDelta .org .apache .iceberg .hadoop . HadoopTables
33+ import shadedForDelta .org .apache .iceberg .hive .{ HiveCatalog , HiveTableOperations }
3434
3535import org .apache .spark .sql .SparkSession
36+ import org .apache .spark .sql .catalyst .catalog .CatalogTable
3637
3738object IcebergConverter {
3839
@@ -60,9 +61,9 @@ class IcebergConverter(spark: SparkSession)
6061 // Save an atomic reference of the snapshot being converted, and the txn that triggered
6162 // resulted in the specified snapshot
6263 protected val currentConversion =
63- new AtomicReference [(Snapshot , Option [ OptimisticTransactionImpl ] )]()
64+ new AtomicReference [(Snapshot , OptimisticTransactionImpl )]()
6465 protected val standbyConversion =
65- new AtomicReference [(Snapshot , Option [ OptimisticTransactionImpl ] )]()
66+ new AtomicReference [(Snapshot , OptimisticTransactionImpl )]()
6667
6768 // Whether our async converter thread is active. We may already have an alive thread that is
6869 // about to shutdown, but in such cases this value should return false.
@@ -81,7 +82,10 @@ class IcebergConverter(spark: SparkSession)
8182 */
8283 override def enqueueSnapshotForConversion (
8384 snapshotToConvert : Snapshot ,
84- txn : Option [OptimisticTransactionImpl ]): Unit = {
85+ txn : OptimisticTransactionImpl ): Unit = {
86+ if (! UniversalFormat .icebergEnabled(snapshotToConvert.metadata)) {
87+ return
88+ }
8589 val log = snapshotToConvert.deltaLog
8690 // Replace any previously queued snapshot
8791 val previouslyQueued = standbyConversion.getAndSet((snapshotToConvert, txn))
@@ -126,7 +130,7 @@ class IcebergConverter(spark: SparkSession)
126130 }
127131
128132 // Get a snapshot to convert from the icebergQueue. Sets the queue to null after.
129- private def getNextSnapshot : (Snapshot , Option [ OptimisticTransactionImpl ] ) =
133+ private def getNextSnapshot : (Snapshot , OptimisticTransactionImpl ) =
130134 asyncThreadLock.synchronized {
131135 val potentialSnapshotAndTxn = standbyConversion.get()
132136 currentConversion.set(potentialSnapshotAndTxn)
@@ -155,21 +159,66 @@ class IcebergConverter(spark: SparkSession)
155159 }
156160 }
157161
162+ /**
163+ * Convert the specified snapshot into Iceberg for the given catalogTable
164+ * @param snapshotToConvert the snapshot that needs to be converted to Iceberg
165+ * @param catalogTable the catalogTable this conversion targets.
166+ * @return Converted Delta version and commit timestamp
167+ */
168+ override def convertSnapshot (
169+ snapshotToConvert : Snapshot , catalogTable : CatalogTable ): Option [(Long , Long )] = {
170+ if (! UniversalFormat .icebergEnabled(snapshotToConvert.metadata)) {
171+ return None
172+ }
173+ convertSnapshot(snapshotToConvert, None , catalogTable)
174+ }
175+
176+ /**
177+ * Convert the specified snapshot into Iceberg when performing an OptimisticTransaction
178+ * on a delta table.
179+ * @param snapshotToConvert the snapshot that needs to be converted to Iceberg
180+ * @param txn the transaction that triggers the conversion. It must
181+ * contain the catalogTable this conversion targets.
182+ * @return Converted Delta version and commit timestamp
183+ */
184+ override def convertSnapshot (
185+ snapshotToConvert : Snapshot , txn : OptimisticTransactionImpl ): Option [(Long , Long )] = {
186+ if (! UniversalFormat .icebergEnabled(snapshotToConvert.metadata)) {
187+ return None
188+ }
189+ txn.catalogTable match {
190+ case Some (table) => convertSnapshot(snapshotToConvert, Some (txn), table)
191+ case _ =>
192+ logWarning(s " CatalogTable for table ${snapshotToConvert.deltaLog.tableId} " +
193+ s " is empty in txn. Skip iceberg conversion. " )
194+ recordDeltaEvent(
195+ snapshotToConvert.deltaLog,
196+ " delta.iceberg.conversion.skipped.emptyCatalogTable" ,
197+ data = Map (
198+ " version" -> snapshotToConvert.version
199+ )
200+ )
201+ None
202+ }
203+ }
204+
158205 /**
159206 * Convert the specified snapshot into Iceberg. NOTE: This operation is blocking. Call
160207 * enqueueSnapshotForConversion to run the operation asynchronously.
161208 * @param snapshotToConvert the snapshot that needs to be converted to Iceberg
162209 * @param txnOpt the OptimisticTransaction that created snapshotToConvert.
163210 * Used as a hint to avoid recomputing old metadata.
211+ * @param catalogTable the catalogTable this conversion targets
164212 * @return Converted Delta version and commit timestamp
165213 */
166- override def convertSnapshot (
214+ private def convertSnapshot (
167215 snapshotToConvert : Snapshot ,
168- txnOpt : Option [OptimisticTransactionImpl ]): Option [(Long , Long )] =
216+ txnOpt : Option [OptimisticTransactionImpl ],
217+ catalogTable : CatalogTable ): Option [(Long , Long )] =
169218 recordFrameProfile(" Delta" , " IcebergConverter.convertSnapshot" ) {
170219 val log = snapshotToConvert.deltaLog
171220 val lastDeltaVersionConverted : Option [Long ] =
172- loadLastDeltaVersionConverted(snapshotToConvert)
221+ loadLastDeltaVersionConverted(snapshotToConvert, catalogTable )
173222 val maxCommitsToConvert =
174223 spark.sessionState.conf.getConf(DeltaSQLConf .ICEBERG_MAX_COMMITS_TO_CONVERT )
175224
@@ -202,8 +251,14 @@ class IcebergConverter(spark: SparkSession)
202251 case (Some (_), None ) => REPLACE_TABLE
203252 case (None , None ) => CREATE_TABLE
204253 }
254+
255+ UniversalFormat .enforceSupportInCatalog(catalogTable, snapshotToConvert.metadata) match {
256+ case Some (updatedTable) => spark.sessionState.catalog.alterTable(updatedTable)
257+ case _ =>
258+ }
259+
205260 val icebergTxn = new IcebergConversionTransaction (
206- log.newDeltaHadoopConf(), snapshotToConvert, tableOp, lastDeltaVersionConverted)
261+ catalogTable, log.newDeltaHadoopConf(), snapshotToConvert, tableOp, lastDeltaVersionConverted)
207262
208263 // Write out the actions taken since the last conversion (or since table creation).
209264 // This is done in batches, with each batch corresponding either to one delta file,
@@ -268,18 +323,10 @@ class IcebergConverter(spark: SparkSession)
268323 Some (snapshotToConvert.version, snapshotToConvert.timestamp)
269324 }
270325
271- override def loadLastDeltaVersionConverted (snapshot : Snapshot ): Option [Long ] =
326+ override def loadLastDeltaVersionConverted (
327+ snapshot : Snapshot , catalogTable : CatalogTable ): Option [Long ] =
272328 recordFrameProfile(" Delta" , " IcebergConverter.loadLastDeltaVersionConverted" ) {
273- val deltaLog = snapshot.deltaLog
274- val hadoopTables = new HadoopTables (deltaLog.newDeltaHadoopConf())
275- if (hadoopTables.exists(deltaLog.dataPath.toString)) {
276- hadoopTables
277- .load(deltaLog.dataPath.toString)
278- .properties()
279- .asScala
280- .get(IcebergConverter .DELTA_VERSION_PROPERTY )
281- .map(_.toLong)
282- } else None
329+ catalogTable.properties.get(IcebergConverter .DELTA_VERSION_PROPERTY ).map(_.toLong)
283330 }
284331
285332 /**
0 commit comments