jasonf20 commented on PR #9323:
URL: https://github.com/apache/iceberg/pull/9323#issuecomment-1859197491

   **Benchmark**
   The following test was run locally just to demonstrate that the difference 
in IO performance is very significant. While the transaction approach IO grows 
linearly with the number of batches, the Streaming API has constant IO 
overhead. 
   
   The test uses just 10 batches, but increasing the batch count will increase 
the difference linearly. 
   
   **code**
   ```scala
   object IcebergUpsert {
   
   
     def main(args: Array[String]): Unit = {
       val catalog = new GlueCatalog()
       catalog.initialize("iceberg", Map.empty[String, String].asJava)
   
       val schema = new Schema(
         IcebergTypes.NestedField.required(1, "id", 
IcebergTypes.StringType.get()),
       );
       val tableName = "streams"
       val tableId = TableIdentifier.of("prod_iceberg", tableName)
       val basePath = s"s3://bucket/ice/tables/${tableName}/"
   
       if (!catalog.tableExists(tableId)) {
         catalog.createTable(tableId, schema, PartitionSpec.unpartitioned(), 
basePath, Map.empty[String, String].asJava)
       }
   
       val table = catalog.loadTable(tableId)
   
       val updateBatches = (1 to 10).par.map(_ => {
         writeFile(basePath, table) -> writeDeleteFile(basePath, table)
       }).seq
   
       println("Starting with rowDelta")
       var startTime = System.currentTimeMillis()
       val transaction = table.newTransaction()
       updateBatches.foreach({
         case (dataFile, deleteFile) =>
           transaction.newRowDelta()
             .addRows(dataFile)
             .addDeletes(deleteFile)
             .commit()
       })
       transaction.commitTransaction()
       println(s"Row delta took ${System.currentTimeMillis() - startTime} 
millis")
       table.refresh()
       println(s"Table has 
${table.currentSnapshot().allManifests(table.io()).size()} manifests")
   
       println("Starting with streaming update")
       startTime = System.currentTimeMillis()
       val update = table.newStreamingUpdate()
       updateBatches.foreach({
         case (dataFile, deleteFile) =>
           update.newBatch().addFile(dataFile).addFile(deleteFile)
       })
       update.commit()
       println(s"Streaming update took ${System.currentTimeMillis() - 
startTime} millis")
       table.refresh()
       println(s"Table has 
${table.currentSnapshot().allManifests(table.io()).size()} manifests")
     }
   
     private def writeFile(basePath: String, table: Table) = {
       val writer = Parquet.writeData(
           table.io().newOutputFile(basePath + UUID.randomUUID().toString + 
".parquet"))
         .forTable(table)
         .overwrite(true)
         .createWriterFunc(GenericParquetWriter.buildWriter)
         .build[data.Record]()
       writer.write(Iterable(GenericRecord.create(table.schema()).copy("id", 
"1")).asJava)
       writer.close()
       writer.toDataFile
     }
   
     private def writeDeleteFile(basePath: String, table: Table) = {
       val writer = Parquet.writeDeletes(
           table.io().newOutputFile(basePath + UUID.randomUUID().toString + 
".parquet"))
         .forTable(table)
         .overwrite(true)
         .createWriterFunc(GenericParquetWriter.buildWriter)
         
.equalityFieldIds(table.schema().columns().asScala.map(_.fieldId().asInstanceOf[Integer]).asJava)
         .buildEqualityWriter[data.Record]()
       writer.write(Iterable(GenericRecord.create(table.schema()).copy("id", 
"1")).asJava)
       writer.close()
       writer.toDeleteFile
     }
   }
   ```
   
   **results**
   ```
   Starting with rowDelta
   2023-12-15 14:40:34,801 8637 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 3558899585328564531 
(BaseRowDelta)
   2023-12-15 14:40:35,219 9055 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=3558899585328564531, sequenceNumber=41, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT2.337073916S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=41}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=41}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=41}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, re
 movedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=33046}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=41}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:36,498 10334 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 616856435651832528 
(BaseRowDelta)
   2023-12-15 14:40:36,917 10753 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=616856435651832528, sequenceNumber=42, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.698149167S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=42}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=42}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=42}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, re
 movedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=33852}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=42}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:38,226 12062 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 4567773841167163715 
(BaseRowDelta)
   2023-12-15 14:40:38,642 12478 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=4567773841167163715, sequenceNumber=43, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.723988375S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=43}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=43}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=43}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, r
 emovedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=34658}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=43}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:39,991 13827 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 4541277781112838040 
(BaseRowDelta)
   2023-12-15 14:40:40,466 14302 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=4541277781112838040, sequenceNumber=44, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.82380625S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=44}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=44}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=44}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, re
 movedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=35464}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=44}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:41,776 15612 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 6741568890407525576 
(BaseRowDelta)
   2023-12-15 14:40:42,184 16020 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=6741568890407525576, sequenceNumber=45, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.717269417S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=45}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=45}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=45}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, r
 emovedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=36270}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=45}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:43,460 17296 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 1018838099119420123 
(BaseRowDelta)
   2023-12-15 14:40:43,877 17713 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=1018838099119420123, sequenceNumber=46, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.692011958S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=46}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=46}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=46}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, r
 emovedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=37076}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=46}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:45,192 19028 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 758872760345785094 
(BaseRowDelta)
   2023-12-15 14:40:45,630 19466 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=758872760345785094, sequenceNumber=47, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.752748416S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=47}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=47}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=47}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, re
 movedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=37882}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=47}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:46,938 20774 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 8330568458953296944 
(BaseRowDelta)
   2023-12-15 14:40:47,345 21181 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=8330568458953296944, sequenceNumber=48, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.714607334S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=48}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=48}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=48}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, r
 emovedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=38688}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=48}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:48,875 22711 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 380337181704891470 
(BaseRowDelta)
   2023-12-15 14:40:49,281 23117 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=380337181704891470, sequenceNumber=49, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.934422667S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=49}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=49}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=49}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, re
 movedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=39494}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=49}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:50,558 24394 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 2626027648794038319 
(BaseRowDelta)
   2023-12-15 14:40:50,960 24796 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=2626027648794038319, sequenceNumber=50, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT1.679267084S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=1}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=50}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=50}, 
addedRecords=CounterResult{unit=COUNT, value=1}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=50}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806}, r
 emovedFilesSizeInBytes=null, totalFilesSizeInBytes=CounterResult{unit=BYTES, 
value=40300}, addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=1}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=50}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   2023-12-15 14:40:52,297 26133 [main] INFO  
o.a.i.BaseMetastoreTableOperations  - Successfully committed to table 
iceberg.prod_iceberg.streams in 1041 ms
   2023-12-15 14:40:52,588 26424 [main] INFO  
o.a.i.BaseMetastoreTableOperations  - Refreshing table metadata from new 
version: 
s3://bucket/ice/tables/streams/metadata/00005-20fd758e-5cb9-431d-95d5-1de0e06bea8f.metadata.json
   Row delta took 24493 millis
   Table has 20 manifests
   Starting with streaming update
   2023-12-15 14:40:59,957 33793 [main] INFO  
o.a.i.BaseMetastoreTableOperations  - Successfully committed to table 
iceberg.prod_iceberg.streams in 1003 ms
   2023-12-15 14:40:59,957 33793 [main] INFO  
org.apache.iceberg.SnapshotProducer  - Committed snapshot 6955191004682032851 
(BaseStreamingUpdate)
   2023-12-15 14:41:00,242 34078 [main] INFO  
o.a.i.BaseMetastoreTableOperations  - Refreshing table metadata from new 
version: 
s3://bucket/ice/tables/streams/metadata/00006-435a312d-615d-4ae0-8c0a-96d60a1aa9d5.metadata.json
   2023-12-15 14:41:01,349 35185 [main] INFO  o.a.i.m.LoggingMetricsReporter  - 
Received metrics report: CommitReport{tableName=iceberg.prod_iceberg.streams, 
snapshotId=6955191004682032851, sequenceNumber=60, operation=overwrite, 
commitMetrics=CommitMetricsResult{totalDuration=TimerResult{timeUnit=NANOSECONDS,
 totalDuration=PT3.728768833S, count=1}, attempts=CounterResult{unit=COUNT, 
value=1}, addedDataFiles=CounterResult{unit=COUNT, value=10}, 
removedDataFiles=null, totalDataFiles=CounterResult{unit=COUNT, value=60}, 
addedDeleteFiles=CounterResult{unit=COUNT, value=10}, 
addedEqualityDeleteFiles=CounterResult{unit=COUNT, value=10}, 
addedPositionalDeleteFiles=null, removedDeleteFiles=null, 
removedEqualityDeleteFiles=null, removedPositionalDeleteFiles=null, 
totalDeleteFiles=CounterResult{unit=COUNT, value=60}, 
addedRecords=CounterResult{unit=COUNT, value=10}, removedRecords=null, 
totalRecords=CounterResult{unit=COUNT, value=60}, 
addedFilesSizeInBytes=CounterResult{unit=BYTES, value=806
 0}, removedFilesSizeInBytes=null, 
totalFilesSizeInBytes=CounterResult{unit=BYTES, value=48360}, 
addedPositionalDeletes=null, removedPositionalDeletes=null, 
totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
addedEqualityDeletes=CounterResult{unit=COUNT, value=10}, 
removedEqualityDeletes=null, totalEqualityDeletes=CounterResult{unit=COUNT, 
value=60}}, metadata={iceberg-version=Apache Iceberg 1.5.0-SNAPSHOT (commit 
0b63a580f84dc38fc891a75fa244724cb9d57385)}}
   Streaming update took 4010 millis
   Table has 22 manifests
   ```
   24 seconds -> 4 seconds


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to