|
17 | 17 | */ |
18 | 18 | package org.apache.beam.sdk.io.iceberg.cdc; |
19 | 19 |
|
20 | | -import java.util.Iterator; |
| 20 | +import java.util.Comparator; |
| 21 | +import java.util.HashSet; |
| 22 | +import java.util.List; |
| 23 | +import java.util.Set; |
| 24 | +import org.apache.beam.sdk.schemas.Schema; |
21 | 25 | import org.apache.beam.sdk.transforms.DoFn; |
22 | 26 | import org.apache.beam.sdk.transforms.join.CoGbkResult; |
23 | 27 | import org.apache.beam.sdk.values.KV; |
24 | 28 | import org.apache.beam.sdk.values.Row; |
25 | 29 | import org.apache.beam.sdk.values.TimestampedValue; |
26 | 30 | import org.apache.beam.sdk.values.TupleTag; |
| 31 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; |
27 | 32 |
|
28 | 33 | /** |
29 | 34 | * Receives inserts and deletes, keyed by snapshot ID and Primary Key, and determines if any updates |
|
34 | 39 | * |
35 | 40 | * <p>Otherwise, records are output as-is: INSERT as INSERT, and DELETE as DELETE. |
36 | 41 | * |
37 | | - * <p>Input elements have their timestamp reified. This is because CoGroupByKey assigns all elements |
38 | | - * in a window with the same timestamp, erasing individual record timestamps. This DoFn preserves it |
39 | | - * by outputting records with their reified timestamps. |
| 42 | + * <p>Note: snapshots written using the Copy-on-Write method will produce tasks where records appear |
| 43 | + * to be deleted then re-inserted. We perform an initial de-duplication and drop these records to |
| 44 | + * avoid mistaking them as no-op updates. |
| 45 | + * |
| 46 | + * <p>Input elements are prepared by reifying their timestamps. This is because CoGroupByKey assigns |
| 47 | + * all elements in a window with the same timestamp, erasing individual record timestamps. This DoFn |
| 48 | + * preserves it by outputting records with their reified timestamps. |
40 | 49 | */ |
41 | 50 | public class ResolveChanges extends DoFn<KV<Row, CoGbkResult>, Row> { |
42 | 51 | public static final TupleTag<TimestampedValue<Row>> DELETES = new TupleTag<>() {}; |
43 | 52 | public static final TupleTag<TimestampedValue<Row>> INSERTS = new TupleTag<>() {}; |
44 | 53 |
|
45 | 54 | @DoFn.ProcessElement |
46 | 55 | public void processElement(@Element KV<Row, CoGbkResult> element, OutputReceiver<Row> out) { |
| 56 | + Set<String> pkFields = new HashSet<>(element.getKey().getSchema().getFieldNames()); |
47 | 57 | CoGbkResult result = element.getValue(); |
48 | 58 |
|
49 | | - // iterables are lazy-loaded from the shuffle service |
50 | | - Iterable<TimestampedValue<Row>> deletes = result.getAll(DELETES); |
51 | | - Iterable<TimestampedValue<Row>> inserts = result.getAll(INSERTS); |
| 59 | + List<TimestampedValue<Row>> deletes = Lists.newArrayList(result.getAll(DELETES)); |
| 60 | + List<TimestampedValue<Row>> inserts = Lists.newArrayList(result.getAll(INSERTS)); |
| 61 | + deletes.sort(Comparator.comparing(TimestampedValue::getTimestamp)); |
| 62 | + inserts.sort(Comparator.comparing(TimestampedValue::getTimestamp)); |
| 63 | + |
| 64 | + boolean[] duplicateDeletes = new boolean[deletes.size()]; |
| 65 | + boolean[] duplicateInserts = new boolean[inserts.size()]; |
52 | 66 |
|
53 | | - boolean hasDeletes = deletes.iterator().hasNext(); |
54 | | - boolean hasInserts = inserts.iterator().hasNext(); |
| 67 | + boolean hasDeletes = !deletes.isEmpty(); |
| 68 | + boolean hasInserts = !inserts.isEmpty(); |
55 | 69 |
|
56 | 70 | if (hasInserts && hasDeletes) { |
57 | 71 | // UPDATE: row ID exists in both streams |
58 | 72 | // - emit all deletes as 'UPDATE_BEFORE', and all inserts as 'UPDATE_AFTER' |
59 | | - // - emit extra inserts as 'UPDATE_AFTER' |
60 | | - // - ignore extra deletes (TODO: double check if this is a good decision) |
61 | | - Iterator<TimestampedValue<Row>> deletesIterator = deletes.iterator(); |
62 | | - Iterator<TimestampedValue<Row>> insertsIterator = inserts.iterator(); |
63 | | - while (deletesIterator.hasNext() && insertsIterator.hasNext()) { |
64 | | - // TODO: output as UPDATE_BEFORE kind |
65 | | - TimestampedValue<Row> updateBefore = deletesIterator.next(); |
66 | | - out.outputWithTimestamp(updateBefore.getValue(), updateBefore.getTimestamp()); |
67 | | - System.out.printf("[BIDIRECTIONAL] -- UpdateBefore%n%s%n", updateBefore); |
68 | | - |
69 | | - // TODO: output as UPDATE_AFTER kind |
70 | | - TimestampedValue<Row> updateAfter = insertsIterator.next(); |
71 | | - out.outputWithTimestamp(updateAfter.getValue(), updateAfter.getTimestamp()); |
72 | | - System.out.printf("[BIDIRECTIONAL] -- UpdateAfter%n%s%n", updateAfter); |
| 73 | + // - emit extra deletes as 'DELETE' |
| 74 | + // - emit extra inserts as 'INSERT' |
| 75 | + |
| 76 | + // First, loop through both deletes and inserts and deduplicate records. |
| 77 | + // Duplicates can occur when an Iceberg writer uses CoW method: |
| 78 | + // Deletes records by rewriting an entire DataFile except for the few records intended for |
| 79 | + // deletion. |
| 80 | + // From our perspective, all the records were deleted, and some were inserted back in. |
| 81 | + // We must ignore these records and not mistake them for "updates". |
| 82 | + for (int d = 0; d < deletes.size(); d++) { |
| 83 | + TimestampedValue<Row> delete = deletes.get(d); |
| 84 | + |
| 85 | + for (int i = 0; i < inserts.size(); i++) { |
| 86 | + if (duplicateInserts[i]) { |
| 87 | + continue; |
| 88 | + } |
| 89 | + |
| 90 | + TimestampedValue<Row> insert = inserts.get(i); |
| 91 | + if (isDuplicate(pkFields, delete.getValue(), insert.getValue())) { |
| 92 | + duplicateDeletes[d] = true; |
| 93 | + duplicateInserts[i] = true; |
| 94 | + System.out.printf("[DEDUPE] -- Ignored CoW record: %s%n", delete); |
| 95 | + break; |
| 96 | + } |
| 97 | + } |
| 98 | + } |
| 99 | + |
| 100 | + // Second, loop through and output UPDATE pairs |
| 101 | + int d = 0; |
| 102 | + int i = 0; |
| 103 | + while (d < deletes.size() && i < inserts.size()) { |
| 104 | + // find next unique delete |
| 105 | + while (d < deletes.size() && duplicateDeletes[d]) { |
| 106 | + d++; |
| 107 | + } |
| 108 | + // find next unique insert |
| 109 | + while (i < inserts.size() && duplicateInserts[i]) { |
| 110 | + i++; |
| 111 | + } |
| 112 | + |
| 113 | + if (d < deletes.size() && i < inserts.size()) { |
| 114 | + // UPDATE pair found. output as UpdateBefore/After |
| 115 | + TimestampedValue<Row> updateBefore = deletes.get(d); |
| 116 | + TimestampedValue<Row> updateAfter = inserts.get(i); |
| 117 | + |
| 118 | + // TODO: output as UPDATE_BEFORE and UPDATE_AFTER kind |
| 119 | + out.outputWithTimestamp(updateBefore.getValue(), updateBefore.getTimestamp()); |
| 120 | + out.outputWithTimestamp(updateAfter.getValue(), updateAfter.getTimestamp()); |
| 121 | + System.out.printf( |
| 122 | + "[BIDIRECTIONAL] -- UpdateBefore:%n\t%s%n\tUpdateAfter%n\t%s%n", |
| 123 | + updateBefore, updateAfter); |
| 124 | + |
| 125 | + d++; |
| 126 | + i++; |
| 127 | + } |
73 | 128 | } |
74 | | - while (insertsIterator.hasNext()) { |
75 | | - // TODO: output as UPDATE_AFTER kind |
76 | | - TimestampedValue<Row> insert = insertsIterator.next(); |
77 | | - out.outputWithTimestamp(insert.getValue(), insert.getTimestamp()); |
78 | | - System.out.printf("[BIDIRECTIONAL] -- Added(extra)%n%s%n", insert); |
| 129 | + |
| 130 | + // Finally, output extra deletes or inserts |
| 131 | + while (d < deletes.size()) { |
| 132 | + // TODO: output as DELETE kind |
| 133 | + if (!duplicateDeletes[d]) { |
| 134 | + TimestampedValue<Row> delete = deletes.get(d); |
| 135 | + out.outputWithTimestamp(delete.getValue(), delete.getTimestamp()); |
| 136 | + System.out.printf("[BIDIRECTIONAL] -- Deleted(extra)%n%s%n", delete); |
| 137 | + } |
| 138 | + d++; |
| 139 | + } |
| 140 | + while (i < inserts.size()) { |
| 141 | + // TODO: output as INSERT kind |
| 142 | + if (!duplicateInserts[i]) { |
| 143 | + TimestampedValue<Row> insert = inserts.get(i); |
| 144 | + out.outputWithTimestamp(insert.getValue(), insert.getTimestamp()); |
| 145 | + System.out.printf("[BIDIRECTIONAL] -- Inserted(extra)%n%s%n", insert); |
| 146 | + } |
| 147 | + i++; |
79 | 148 | } |
80 | 149 | } else if (hasInserts) { |
81 | 150 | // INSERT only |
82 | 151 | for (TimestampedValue<Row> rec : inserts) { |
83 | | - System.out.printf("[UNIDIRECTIONAL] -- Added%n%s%n", rec); |
| 152 | + System.out.printf("[BIDIRECTIONAL (only inserts)] -- Added%n%s%n", rec); |
84 | 153 | out.outputWithTimestamp(rec.getValue(), rec.getTimestamp()); |
85 | 154 | } |
86 | 155 | } else if (hasDeletes) { |
87 | 156 | // DELETE only |
88 | 157 | for (TimestampedValue<Row> rec : deletes) { |
89 | 158 | // TODO: output as DELETE kind |
90 | | - System.out.printf("[UNIDIRECTIONAL] -- Deleted%n%s%n", rec); |
| 159 | + System.out.printf("[BIDIRECTIONAL (only deletes)] -- Deleted%n%s%n", rec); |
91 | 160 | out.outputWithTimestamp(rec.getValue(), rec.getTimestamp()); |
92 | 161 | } |
93 | 162 | } |
94 | 163 | } |
| 164 | + |
| 165 | + /** Compares both records and checks whether they are duplicates or not. */ |
| 166 | + private static boolean isDuplicate(Set<String> pkFields, Row delete, Row insert) { |
| 167 | + Schema schema = insert.getSchema(); |
| 168 | + for (String field : insert.getSchema().getFieldNames()) { |
| 169 | + if (pkFields.contains(field)) { |
| 170 | + // these records are grouped by Primary Key, so we already know PK values are equal |
| 171 | + continue; |
| 172 | + } |
| 173 | + // return early if two values are not equal |
| 174 | + if (!Row.Equals.deepEquals( |
| 175 | + insert.getValue(field), delete.getValue(field), schema.getField(field).getType())) { |
| 176 | + return false; |
| 177 | + } |
| 178 | + } |
| 179 | + return true; |
| 180 | + } |
95 | 181 | } |
0 commit comments