@@ -23,16 +23,17 @@ import scala.collection.JavaConverters._
23
23
import scala .util .control .Breaks ._
24
24
import scala .util .control .NonFatal
25
25
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 }
27
27
import org .apache .spark .sql .delta .actions .{Action , AddFile , CommitInfo , RemoveFile }
28
28
import org .apache .spark .sql .delta .hooks .IcebergConverterHook
29
29
import org .apache .spark .sql .delta .metering .DeltaLogging
30
30
import org .apache .spark .sql .delta .sources .DeltaSQLConf
31
31
import org .apache .commons .lang3 .exception .ExceptionUtils
32
32
import org .apache .hadoop .fs .Path
33
- import shadedForDelta .org .apache .iceberg .hadoop . HadoopTables
33
+ import shadedForDelta .org .apache .iceberg .hive .{ HiveCatalog , HiveTableOperations }
34
34
35
35
import org .apache .spark .sql .SparkSession
36
+ import org .apache .spark .sql .catalyst .catalog .CatalogTable
36
37
37
38
object IcebergConverter {
38
39
@@ -60,9 +61,9 @@ class IcebergConverter(spark: SparkSession)
60
61
// Save an atomic reference of the snapshot being converted, and the txn that triggered
61
62
// resulted in the specified snapshot
62
63
protected val currentConversion =
63
- new AtomicReference [(Snapshot , Option [ OptimisticTransactionImpl ] )]()
64
+ new AtomicReference [(Snapshot , OptimisticTransactionImpl )]()
64
65
protected val standbyConversion =
65
- new AtomicReference [(Snapshot , Option [ OptimisticTransactionImpl ] )]()
66
+ new AtomicReference [(Snapshot , OptimisticTransactionImpl )]()
66
67
67
68
// Whether our async converter thread is active. We may already have an alive thread that is
68
69
// about to shutdown, but in such cases this value should return false.
@@ -81,7 +82,10 @@ class IcebergConverter(spark: SparkSession)
81
82
*/
82
83
override def enqueueSnapshotForConversion (
83
84
snapshotToConvert : Snapshot ,
84
- txn : Option [OptimisticTransactionImpl ]): Unit = {
85
+ txn : OptimisticTransactionImpl ): Unit = {
86
+ if (! UniversalFormat .icebergEnabled(snapshotToConvert.metadata)) {
87
+ return
88
+ }
85
89
val log = snapshotToConvert.deltaLog
86
90
// Replace any previously queued snapshot
87
91
val previouslyQueued = standbyConversion.getAndSet((snapshotToConvert, txn))
@@ -126,7 +130,7 @@ class IcebergConverter(spark: SparkSession)
126
130
}
127
131
128
132
// 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 ) =
130
134
asyncThreadLock.synchronized {
131
135
val potentialSnapshotAndTxn = standbyConversion.get()
132
136
currentConversion.set(potentialSnapshotAndTxn)
@@ -155,21 +159,66 @@ class IcebergConverter(spark: SparkSession)
155
159
}
156
160
}
157
161
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
+
158
205
/**
159
206
* Convert the specified snapshot into Iceberg. NOTE: This operation is blocking. Call
160
207
* enqueueSnapshotForConversion to run the operation asynchronously.
161
208
* @param snapshotToConvert the snapshot that needs to be converted to Iceberg
162
209
* @param txnOpt the OptimisticTransaction that created snapshotToConvert.
163
210
* Used as a hint to avoid recomputing old metadata.
211
+ * @param catalogTable the catalogTable this conversion targets
164
212
* @return Converted Delta version and commit timestamp
165
213
*/
166
- override def convertSnapshot (
214
+ private def convertSnapshot (
167
215
snapshotToConvert : Snapshot ,
168
- txnOpt : Option [OptimisticTransactionImpl ]): Option [(Long , Long )] =
216
+ txnOpt : Option [OptimisticTransactionImpl ],
217
+ catalogTable : CatalogTable ): Option [(Long , Long )] =
169
218
recordFrameProfile(" Delta" , " IcebergConverter.convertSnapshot" ) {
170
219
val log = snapshotToConvert.deltaLog
171
220
val lastDeltaVersionConverted : Option [Long ] =
172
- loadLastDeltaVersionConverted(snapshotToConvert)
221
+ loadLastDeltaVersionConverted(snapshotToConvert, catalogTable )
173
222
val maxCommitsToConvert =
174
223
spark.sessionState.conf.getConf(DeltaSQLConf .ICEBERG_MAX_COMMITS_TO_CONVERT )
175
224
@@ -202,8 +251,14 @@ class IcebergConverter(spark: SparkSession)
202
251
case (Some (_), None ) => REPLACE_TABLE
203
252
case (None , None ) => CREATE_TABLE
204
253
}
254
+
255
+ UniversalFormat .enforceSupportInCatalog(catalogTable, snapshotToConvert.metadata) match {
256
+ case Some (updatedTable) => spark.sessionState.catalog.alterTable(updatedTable)
257
+ case _ =>
258
+ }
259
+
205
260
val icebergTxn = new IcebergConversionTransaction (
206
- log.newDeltaHadoopConf(), snapshotToConvert, tableOp, lastDeltaVersionConverted)
261
+ catalogTable, log.newDeltaHadoopConf(), snapshotToConvert, tableOp, lastDeltaVersionConverted)
207
262
208
263
// Write out the actions taken since the last conversion (or since table creation).
209
264
// This is done in batches, with each batch corresponding either to one delta file,
@@ -268,18 +323,10 @@ class IcebergConverter(spark: SparkSession)
268
323
Some (snapshotToConvert.version, snapshotToConvert.timestamp)
269
324
}
270
325
271
- override def loadLastDeltaVersionConverted (snapshot : Snapshot ): Option [Long ] =
326
+ override def loadLastDeltaVersionConverted (
327
+ snapshot : Snapshot , catalogTable : CatalogTable ): Option [Long ] =
272
328
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)
283
330
}
284
331
285
332
/**
0 commit comments