@@ -66,8 +66,8 @@ public class ReadFromChangelogs<OutT>
6666 private transient StructProjection recordIdProjection ;
6767 private transient org .apache .iceberg .Schema recordIdSchema ;
6868 private final Schema beamRowSchema ;
69- private final Schema rowIdWithOrdinalBeamSchema ;
70- private static final String ORDINAL_FIELD = "__beam__changelog__ordinal__ " ;
69+ private final Schema rowAndSnapshotIDBeamSchema ;
70+ private static final String SNAPSHOT_FIELD = "__beam__changelog__snapshot__id__ " ;
7171
7272 private ReadFromChangelogs (IcebergScanConfig scanConfig , boolean keyedOutput ) {
7373 this .scanConfig = scanConfig ;
@@ -78,13 +78,7 @@ private ReadFromChangelogs(IcebergScanConfig scanConfig, boolean keyedOutput) {
7878 this .recordIdSchema = recordSchema .select (recordSchema .identifierFieldNames ());
7979 this .recordIdProjection = StructProjection .create (recordSchema , recordIdSchema );
8080
81- Schema rowIdBeamSchema = icebergSchemaToBeamSchema (recordIdSchema );
82- List <Schema .Field > fields =
83- ImmutableList .<Schema .Field >builder ()
84- .add (Schema .Field .of (ORDINAL_FIELD , Schema .FieldType .INT32 ))
85- .addAll (rowIdBeamSchema .getFields ())
86- .build ();
87- this .rowIdWithOrdinalBeamSchema = new Schema (fields );
81+ this .rowAndSnapshotIDBeamSchema = rowAndSnapshotIDBeamSchema (scanConfig );
8882 }
8983
9084 static ReadFromChangelogs <Row > of (IcebergScanConfig scanConfig ) {
@@ -100,19 +94,24 @@ static ReadFromChangelogs<KV<Row, Row>> withKeyedOutput(IcebergScanConfig scanCo
10094 * schema's identifier fields.
10195 */
10296 static KvCoder <Row , Row > keyedOutputCoder (IcebergScanConfig scanConfig ) {
97+ org .apache .iceberg .Schema recordSchema = scanConfig .getProjectedSchema ();
98+ Schema rowAndSnapshotIDBeamSchema = rowAndSnapshotIDBeamSchema (scanConfig );
99+ return KvCoder .of (
100+ SchemaCoder .of (rowAndSnapshotIDBeamSchema ),
101+ SchemaCoder .of (icebergSchemaToBeamSchema (recordSchema )));
102+ }
103+
104+ private static Schema rowAndSnapshotIDBeamSchema (IcebergScanConfig scanConfig ) {
103105 org .apache .iceberg .Schema recordSchema = scanConfig .getProjectedSchema ();
104106 org .apache .iceberg .Schema recordIdSchema =
105- recordSchema .select (recordSchema .identifierFieldNames ());
107+ recordSchema .select (recordSchema .identifierFieldNames ());
106108 Schema rowIdBeamSchema = icebergSchemaToBeamSchema (recordIdSchema );
107109 List <Schema .Field > fields =
108- ImmutableList .<Schema .Field >builder ()
109- .add (Schema .Field .of (ORDINAL_FIELD , Schema .FieldType .INT32 ))
110- .addAll (rowIdBeamSchema .getFields ())
111- .build ();
112- Schema rowIdWithOrdinalBeamSchema = new Schema (fields );
113- return KvCoder .of (
114- SchemaCoder .of (rowIdWithOrdinalBeamSchema ),
115- SchemaCoder .of (icebergSchemaToBeamSchema (recordSchema )));
110+ ImmutableList .<Schema .Field >builder ()
111+ .add (Schema .Field .of (SNAPSHOT_FIELD , Schema .FieldType .INT64 ))
112+ .addAll (rowIdBeamSchema .getFields ())
113+ .build ();
114+ return new Schema (fields );
116115 }
117116
118117 @ Setup
@@ -164,12 +163,16 @@ private void processAddedRowsTask(
164163 try (CloseableIterable <Record > fullIterable = ReadUtils .createReader (task , table , scanConfig )) {
165164 DeleteFilter <Record > deleteFilter =
166165 ReadUtils .genericDeleteFilter (
167- table , scanConfig , task .getDataFile ().getPath (), task .getExistingDeletes ());
166+ table , scanConfig , task .getDataFile ().getPath (), task .getAddedDeletes ());
168167 CloseableIterable <Record > filtered = deleteFilter .filter (fullIterable );
169168
170169 for (Record rec : filtered ) {
171170 outputRecord (
172- rec , outputReceiver , task .getOrdinal (), task .getTimestampMillis (), KEYED_INSERTS );
171+ rec ,
172+ outputReceiver ,
173+ task .getCommitSnapshotId (),
174+ task .getTimestampMillis (),
175+ KEYED_INSERTS );
173176 }
174177 }
175178 numAddedRowsScanTasksCompleted .inc ();
@@ -192,7 +195,11 @@ private void processDeletedRowsTask(
192195 for (Record rec : newlyDeletedRecords ) {
193196 // TODO: output with DELETE kind
194197 outputRecord (
195- rec , outputReceiver , task .getOrdinal (), task .getTimestampMillis (), KEYED_DELETES );
198+ rec ,
199+ outputReceiver ,
200+ task .getCommitSnapshotId (),
201+ task .getTimestampMillis (),
202+ KEYED_DELETES );
196203 }
197204 }
198205 numDeletedRowsScanTasksCompleted .inc ();
@@ -209,7 +216,11 @@ private void processDeletedFileTask(
209216 for (Record rec : filtered ) {
210217 // TODO: output with DELETE kind
211218 outputRecord (
212- rec , outputReceiver , task .getOrdinal (), task .getTimestampMillis (), KEYED_DELETES );
219+ rec ,
220+ outputReceiver ,
221+ task .getCommitSnapshotId (),
222+ task .getTimestampMillis (),
223+ KEYED_DELETES );
213224 }
214225 }
215226 numDeletedDataFileScanTasksCompleted .inc ();
@@ -218,35 +229,38 @@ private void processDeletedFileTask(
218229 private void outputRecord (
219230 Record rec ,
220231 MultiOutputReceiver outputReceiver ,
221- int ordinal ,
232+ long snapshotId ,
222233 long timestampMillis ,
223234 TupleTag <KV <Row , Row >> keyedTag ) {
224235 Row row = IcebergUtils .icebergRecordToBeamRow (beamRowSchema , rec );
225236 Instant timestamp = Instant .ofEpochMilli (timestampMillis );
226237 if (keyedOutput ) { // slow path
227238 StructProjection recId = recordIdProjection .wrap (rec );
228- // Create a Row ID consisting of record ID columns and the changelog task's ordinal #
229- Row id = structToBeamRow (ordinal , recId , recordIdSchema , rowIdWithOrdinalBeamSchema );
239+ // Create a Row ID consisting of:
240+ // 1. the task's commit snapshot ID
241+ // 2. the record ID column values
242+ // This is needed to sufficiently distinguish a record change
243+ Row id = structToBeamRow (snapshotId , recId , recordIdSchema , rowAndSnapshotIDBeamSchema );
230244 outputReceiver .get (keyedTag ).outputWithTimestamp (KV .of (id , row ), timestamp );
231245 } else { // fast path
232- System .out .printf ("[UNIFORM] -- Output(%s, %s)\n %s%n" , ordinal , timestamp , row );
246+ System .out .printf ("[UNIFORM] -- Output(%s, %s)\n %s%n" , snapshotId , timestamp , row );
233247 outputReceiver .get (UNIFORM_ROWS ).outputWithTimestamp (row , timestamp );
234248 }
235249 }
236250
237251 public static Row structToBeamRow (
238- int ordinal , StructLike struct , org .apache .iceberg .Schema schema , Schema beamSchema ) {
252+ long snapshotId , StructLike struct , org .apache .iceberg .Schema schema , Schema beamSchema ) {
239253 ImmutableMap .Builder <String , Object > values = ImmutableMap .builder ();
240254 List <Types .NestedField > columns = schema .columns ();
241255 for (Types .NestedField column : columns ) {
242256 String name = column .name ();
243257 Object value = schema .accessorForField (column .fieldId ()).get (struct );
244258 values .put (name , value );
245259 }
246- // Include ordinal as part of the row ID.
260+ // Include snapshot ID as part of the row ID.
247261 // This is essential to ensure that the downstream ReconcileChanges compares rows
248262 // within the same operation.
249- values .put (ORDINAL_FIELD , ordinal );
263+ values .put (SNAPSHOT_FIELD , snapshotId );
250264 return Row .withSchema (beamSchema ).withFieldValues (values .build ()).build ();
251265 }
252266
0 commit comments