From 1b590592b394450029de1cd19d448a0a9cb3a2e5 Mon Sep 17 00:00:00 2001 From: Sungjin Yook Date: Tue, 2 Jul 2024 08:57:24 -0500 Subject: [PATCH 01/29] txid in Debezium cdc metadata --- .../connect/transforms/CdcConstants.java | 2 ++ .../connect/transforms/DebeziumTransform.java | 17 ++++++++++++++++- .../transforms/DebeziumTransformTest.java | 1 + 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CdcConstants.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CdcConstants.java index 25b997f8..c5c01d1c 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CdcConstants.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CdcConstants.java @@ -31,4 +31,6 @@ public interface CdcConstants { String COL_SOURCE = "source"; String COL_TARGET = "target"; String COL_KEY = "key"; + + String COL_TXID = "txid"; } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index 1e79e211..39683882 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -179,22 +179,31 @@ private String mapOperation(String originalOp) { private void setTableAndTargetFromSourceStruct(Struct source, Struct cdcMetadata) { String db; + Long txid = null; + if (source.schema().field("schema") != null) { // prefer schema if present, e.g. for Postgres db = source.getString("schema"); } else { db = source.getString("db"); } + String table = source.getString("table"); + if (source.schema().field("txId") != null) { + txid = source.getInt64("txId"); + } + cdcMetadata.put(CdcConstants.COL_SOURCE, db + "." + table); cdcMetadata.put(CdcConstants.COL_TARGET, target(db, table)); + cdcMetadata.put(CdcConstants.COL_TXID, txid); } private void setTableAndTargetFromSourceMap(Object source, Map cdcMetadata) { Map map = Requirements.requireMap(source, "Debezium transform"); String db; + Long txid = null; if (map.containsKey("schema")) { // prefer schema if present, e.g. for Postgres db = map.get("schema").toString(); @@ -203,8 +212,13 @@ private void setTableAndTargetFromSourceMap(Object source, Map c } String table = map.get("table").toString(); + if (map.containsKey("txId")) { + txid = Long.valueOf(map.get("txId").toString()); + } + cdcMetadata.put(CdcConstants.COL_SOURCE, db + "." + table); cdcMetadata.put(CdcConstants.COL_TARGET, target(db, table)); + cdcMetadata.put(CdcConstants.COL_TXID, txid); } private String target(String db, String table) { @@ -220,7 +234,8 @@ private Schema makeCdcSchema(Schema keySchema) { .field(CdcConstants.COL_TS, Timestamp.SCHEMA) .field(CdcConstants.COL_OFFSET, Schema.OPTIONAL_INT64_SCHEMA) .field(CdcConstants.COL_SOURCE, Schema.STRING_SCHEMA) - .field(CdcConstants.COL_TARGET, Schema.STRING_SCHEMA); + .field(CdcConstants.COL_TARGET, Schema.STRING_SCHEMA) + .field(CdcConstants.COL_TXID, Schema.OPTIONAL_INT64_SCHEMA); if (keySchema != null) { builder.field(CdcConstants.COL_KEY, keySchema); diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index c9f34848..d90af5d7 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -57,6 +57,7 @@ public class DebeziumTransformTest { .field("source", SOURCE_SCHEMA) .field("before", ROW_SCHEMA) .field("after", ROW_SCHEMA) + .field("txid", Schema.INT64_SCHEMA) .build(); @Test From ff2702d7d65be46ac58dd23b6492a2ddb5684d30 Mon Sep 17 00:00:00 2001 From: dtobon Date: Thu, 18 Jul 2024 11:57:52 -0400 Subject: [PATCH 02/29] WIP --- .../iceberg/connect/channel/Coordinator.java | 51 ++++++++++++++----- .../connect/channel/TxIdValidUntilEvent.java | 31 +++++++++++ 2 files changed, 68 insertions(+), 14 deletions(-) create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index b7204c85..301f63c0 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -27,10 +27,7 @@ import java.io.UncheckedIOException; import java.time.Duration; import java.time.OffsetDateTime; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Optional; +import java.util.*; import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.iceberg.AppendFiles; @@ -65,6 +62,10 @@ public class Coordinator extends Channel implements AutoCloseable { private static final String VTTS_SNAPSHOT_PROP = "kafka.connect.vtts"; private static final Duration POLL_DURATION = Duration.ofMillis(1000); + private static final String TXID_VALID_UNTIL_PROP = "txid_valid_until"; + private final Map lastProcessedTxIdPerPartition = new HashMap<>(); + private final Map highestTxIdPerPartition = new HashMap<>(); + private final Catalog catalog; private final IcebergSinkConfig config; private final int totalPartitionCount; @@ -98,10 +99,12 @@ public void process() { // send out begin commit commitState.startNewCommit(); LOG.info("Started new commit with commit-id={}", commitState.currentCommitId().toString()); - Event event = - new Event(config.controlGroupId(), new StartCommit(commitState.currentCommitId())); - send(event); - LOG.info("Sent workers commit trigger with commit-id={}", commitState.currentCommitId().toString()); + // Iterate over each partition and its last processed txId + lastProcessedTxIdPerPartition.forEach((partition, txId) -> { + TxIdValidUntilEvent event = new TxIdValidUntilEvent(config.controlGroupId(), new StartCommit(commitState.currentCommitId()), partition, txId); + send(event); + LOG.info("Sent workers commit trigger for partition {} with commit-id={}", partition, commitState.currentCommitId().toString()); + }); } @@ -115,6 +118,9 @@ public void process() { private boolean receive(Envelope envelope) { switch (envelope.event().type()) { case DATA_WRITTEN: + TxIdValidUntilEvent dataEvent = (TxIdValidUntilEvent) envelope.event(); + highestTxIdPerPartition.merge(dataEvent.getPartition(), dataEvent.getTxId(), Math::max); + lastProcessedTxIdPerPartition.put(dataEvent.getPartition(), dataEvent.getTxId()); commitState.addResponse(envelope); return true; case DATA_COMPLETE: @@ -139,6 +145,8 @@ private void commit(boolean partialCommit) { } private void doCommit(boolean partialCommit) { + String txidValidUntilStr = Long.toString(calculateTxidValidUntil()); + Map> commitMap = commitState.tableCommitMap(); String offsetsJson = offsetsJson(); @@ -147,11 +155,7 @@ private void doCommit(boolean partialCommit) { Tasks.foreach(commitMap.entrySet()) .executeWith(exec) .stopOnFailure() - .run( - entry -> { - commitToTable(entry.getKey(), entry.getValue(), offsetsJson, vtts); - }); - + .run(entry -> commitToTable(entry.getKey(), entry.getValue(), offsetsJson, vtts, txidValidUntilStr)); // we should only get here if all tables committed successfully... commitConsumerOffsets(); commitState.clearResponses(); @@ -179,7 +183,8 @@ private void commitToTable( TableIdentifier tableIdentifier, List envelopeList, String offsetsJson, - OffsetDateTime vtts) { + OffsetDateTime vtts, + String txidValidUntilStr) { Table table; try { table = catalog.loadTable(tableIdentifier); @@ -237,6 +242,7 @@ private void commitToTable( if (vtts != null) { appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } + appendOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); } appendOp.commit(); @@ -251,6 +257,7 @@ private void commitToTable( if (vtts != null) { deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } + deltaOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); deltaOp.commit(); @@ -276,6 +283,22 @@ private void commitToTable( } } + // Calculate the highest txid_valid_until value across all partitions + private long calculateTxidValidUntil() { + // If the map is empty, return 0 as no transactions are guaranteed to be completed + if (highestTxIdPerPartition.isEmpty()) { + return 0L; + } + + // Find the minimum value in the map, as it represents the highest transaction ID + // that is guaranteed to be completed across all partitions + long minValue = Collections.min(highestTxIdPerPartition.values()); + + // Subtract 1 from the minimum value to get the last guaranteed completed transaction ID + // If minValue is 1, then there are no completed transactions, so return 0 + return minValue > 1 ? minValue - 1 : 0; + } + private Snapshot latestSnapshot(Table table, String branch) { if (branch == null) { return table.currentSnapshot(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java new file mode 100644 index 00000000..3cb41293 --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java @@ -0,0 +1,31 @@ +package io.tabular.iceberg.connect.channel; + +import org.apache.avro.Schema; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Payload; + +public class TxIdValidUntilEvent extends Event { + + private final int partition; + private final long txId; + + public TxIdValidUntilEvent(Schema schema, int partition, long txId) { + super(schema); + this.partition = partition; + this.txId = txId; + } + + public TxIdValidUntilEvent(String groupId, Payload payload, int partition, long txId) { + super(groupId, payload); + this.partition = partition; + this.txId = txId; + } + + public int getPartition() { + return partition; + } + + public long getTxId() { + return txId; + } +} \ No newline at end of file From d2db2ba53f668a00ccad462392313f7dfd04327c Mon Sep 17 00:00:00 2001 From: dtobon Date: Fri, 19 Jul 2024 16:58:58 -0400 Subject: [PATCH 03/29] WIP --- .../iceberg/connect/channel/Coordinator.java | 36 ++++++++++++------- .../connect/channel/TxIdValidUntilEvent.java | 31 ---------------- 2 files changed, 23 insertions(+), 44 deletions(-) delete mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 301f63c0..b10a4ca1 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -27,7 +27,11 @@ import java.io.UncheckedIOException; import java.time.Duration; import java.time.OffsetDateTime; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.iceberg.AppendFiles; @@ -42,11 +46,13 @@ import org.apache.iceberg.connect.events.CommitComplete; import org.apache.iceberg.connect.events.CommitToTable; import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Payload; import org.apache.iceberg.connect.events.StartCommit; import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.kafka.clients.admin.MemberDescription; @@ -63,8 +69,8 @@ public class Coordinator extends Channel implements AutoCloseable { private static final Duration POLL_DURATION = Duration.ofMillis(1000); private static final String TXID_VALID_UNTIL_PROP = "txid_valid_until"; - private final Map lastProcessedTxIdPerPartition = new HashMap<>(); - private final Map highestTxIdPerPartition = new HashMap<>(); + private final Map lastProcessedTxIdPerPartition = Maps.newHashMap(); + private final Map highestTxIdPerPartition = Maps.newHashMap(); private final Catalog catalog; private final IcebergSinkConfig config; @@ -99,13 +105,10 @@ public void process() { // send out begin commit commitState.startNewCommit(); LOG.info("Started new commit with commit-id={}", commitState.currentCommitId().toString()); - // Iterate over each partition and its last processed txId - lastProcessedTxIdPerPartition.forEach((partition, txId) -> { - TxIdValidUntilEvent event = new TxIdValidUntilEvent(config.controlGroupId(), new StartCommit(commitState.currentCommitId()), partition, txId); - send(event); - LOG.info("Sent workers commit trigger for partition {} with commit-id={}", partition, commitState.currentCommitId().toString()); - }); - + Event event = + new Event(config.controlGroupId(), new StartCommit(commitState.currentCommitId())); + send(event); + LOG.info("Sent workers commit trigger with commit-id={}", commitState.currentCommitId().toString()); } consumeAvailable(POLL_DURATION, this::receive); @@ -118,9 +121,10 @@ public void process() { private boolean receive(Envelope envelope) { switch (envelope.event().type()) { case DATA_WRITTEN: - TxIdValidUntilEvent dataEvent = (TxIdValidUntilEvent) envelope.event(); - highestTxIdPerPartition.merge(dataEvent.getPartition(), dataEvent.getTxId(), Math::max); - lastProcessedTxIdPerPartition.put(dataEvent.getPartition(), dataEvent.getTxId()); + // Get the transaction id (txId) placed in the CDC metadata + Long txId = extractTxId(envelope); + highestTxIdPerPartition.merge(envelope.partition(), txId , Math::max); + lastProcessedTxIdPerPartition.put(envelope.partition(),txId); commitState.addResponse(envelope); return true; case DATA_COMPLETE: @@ -299,6 +303,12 @@ private long calculateTxidValidUntil() { return minValue > 1 ? minValue - 1 : 0; } + public Long extractTxId(Envelope envelope) { + Payload payload = envelope.event().payload(); + // TODO figure out how to parse the txIc from the payload + return 0L; + } + private Snapshot latestSnapshot(Table table, String branch) { if (branch == null) { return table.currentSnapshot(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java deleted file mode 100644 index 3cb41293..00000000 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TxIdValidUntilEvent.java +++ /dev/null @@ -1,31 +0,0 @@ -package io.tabular.iceberg.connect.channel; - -import org.apache.avro.Schema; -import org.apache.iceberg.connect.events.Event; -import org.apache.iceberg.connect.events.Payload; - -public class TxIdValidUntilEvent extends Event { - - private final int partition; - private final long txId; - - public TxIdValidUntilEvent(Schema schema, int partition, long txId) { - super(schema); - this.partition = partition; - this.txId = txId; - } - - public TxIdValidUntilEvent(String groupId, Payload payload, int partition, long txId) { - super(groupId, payload); - this.partition = partition; - this.txId = txId; - } - - public int getPartition() { - return partition; - } - - public long getTxId() { - return txId; - } -} \ No newline at end of file From 5bd5711ec788e1f7799e7c9d852fe88a51c89cad Mon Sep 17 00:00:00 2001 From: dtobon Date: Tue, 23 Jul 2024 09:06:55 -0400 Subject: [PATCH 04/29] WIP --- .../iceberg/connect/channel/Channel.java | 31 +++++++++++++++++++ .../iceberg/connect/channel/Coordinator.java | 27 +++------------- 2 files changed, 36 insertions(+), 22 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java index d6fc5e6e..46159a76 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java @@ -20,6 +20,8 @@ import static java.util.stream.Collectors.toList; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.data.Offset; import java.time.Duration; @@ -56,6 +58,7 @@ public abstract class Channel { private final Admin admin; private final Map controlTopicOffsets = Maps.newHashMap(); private final String producerId; + private final Map highestTxIdPerPartition = Maps.newHashMap(); private final EventDecoder eventDecoder; @@ -124,6 +127,12 @@ protected void consumeAvailable(Duration pollDuration, Function { + Long txId = extractTxId(record.value()); + if (txId != null) { + highestTxIdPerPartition.merge(record.partition(), txId, Math::max); + } else { + LOG.warn("txId is null for record: {}", record.key()); + } // the consumer stores the offsets that corresponds to the next record to consume, // so increment the record offset by one controlTopicOffsets.put(record.partition(), record.offset() + 1); @@ -146,6 +155,10 @@ protected Map controlTopicOffsets() { return controlTopicOffsets; } + protected Map highestTxIdPerPartition() { + return highestTxIdPerPartition; + } + protected void commitConsumerOffsets() { Map offsetsToCommit = Maps.newHashMap(); controlTopicOffsets() @@ -165,4 +178,22 @@ public void stop() { consumer.close(); admin.close(); } + + public Long extractTxId(byte[] data) { + ObjectMapper objectMapper = new ObjectMapper(); + try { + String jsonPayload = objectMapper.writeValueAsString(data); + JsonNode rootNode = objectMapper.readTree(jsonPayload); + JsonNode cdcNode = rootNode.path("_cdc"); + if (!cdcNode.isMissingNode()) { + JsonNode txIdNode = cdcNode.path("txId"); + if (!txIdNode.isMissingNode()) { + return txIdNode.asLong(); + } + } + } catch (Exception e) { + LOG.warn("Failed to extract txId from payload", e); + } + return null; + } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index b10a4ca1..84dd8fd8 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -46,13 +46,11 @@ import org.apache.iceberg.connect.events.CommitComplete; import org.apache.iceberg.connect.events.CommitToTable; import org.apache.iceberg.connect.events.Event; -import org.apache.iceberg.connect.events.Payload; import org.apache.iceberg.connect.events.StartCommit; import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.kafka.clients.admin.MemberDescription; @@ -67,10 +65,7 @@ public class Coordinator extends Channel implements AutoCloseable { private static final String COMMIT_ID_SNAPSHOT_PROP = "kafka.connect.commit-id"; private static final String VTTS_SNAPSHOT_PROP = "kafka.connect.vtts"; private static final Duration POLL_DURATION = Duration.ofMillis(1000); - - private static final String TXID_VALID_UNTIL_PROP = "txid_valid_until"; - private final Map lastProcessedTxIdPerPartition = Maps.newHashMap(); - private final Map highestTxIdPerPartition = Maps.newHashMap(); + private static final String TXID_VALID_UNTIL_PROP = "txid_valid_until"; private final Catalog catalog; private final IcebergSinkConfig config; @@ -121,10 +116,6 @@ public void process() { private boolean receive(Envelope envelope) { switch (envelope.event().type()) { case DATA_WRITTEN: - // Get the transaction id (txId) placed in the CDC metadata - Long txId = extractTxId(envelope); - highestTxIdPerPartition.merge(envelope.partition(), txId , Math::max); - lastProcessedTxIdPerPartition.put(envelope.partition(),txId); commitState.addResponse(envelope); return true; case DATA_COMPLETE: @@ -246,7 +237,7 @@ private void commitToTable( if (vtts != null) { appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - appendOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); + appendOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); } appendOp.commit(); @@ -261,7 +252,7 @@ private void commitToTable( if (vtts != null) { deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - deltaOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); + deltaOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); deltaOp.commit(); @@ -287,28 +278,20 @@ private void commitToTable( } } - // Calculate the highest txid_valid_until value across all partitions private long calculateTxidValidUntil() { - // If the map is empty, return 0 as no transactions are guaranteed to be completed - if (highestTxIdPerPartition.isEmpty()) { + if (highestTxIdPerPartition().isEmpty()) { return 0L; } // Find the minimum value in the map, as it represents the highest transaction ID // that is guaranteed to be completed across all partitions - long minValue = Collections.min(highestTxIdPerPartition.values()); + long minValue = Collections.min(highestTxIdPerPartition().values()); // Subtract 1 from the minimum value to get the last guaranteed completed transaction ID // If minValue is 1, then there are no completed transactions, so return 0 return minValue > 1 ? minValue - 1 : 0; } - public Long extractTxId(Envelope envelope) { - Payload payload = envelope.event().payload(); - // TODO figure out how to parse the txIc from the payload - return 0L; - } - private Snapshot latestSnapshot(Table table, String branch) { if (branch == null) { return table.currentSnapshot(); From 52f6fd9142b399f4d0b884159a26a1d202cc0dc7 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 24 Jul 2024 14:27:55 -0400 Subject: [PATCH 05/29] Add transaction consistency --- .../iceberg/connect/channel/Channel.java | 28 +---------- .../iceberg/connect/channel/Committable.java | 8 ++- .../connect/channel/CommitterImpl.java | 9 ++-- .../iceberg/connect/channel/Coordinator.java | 35 ++++++------- .../iceberg/connect/channel/Worker.java | 12 ++++- .../connect/data/TransactionEvent.java | 49 +++++++++++++++++++ .../iceberg/connect/data/Utilities.java | 24 +++++++++ .../connect/channel/CommitterImplTest.java | 4 +- .../iceberg/connect/channel/WorkerTest.java | 21 ++++++++ 9 files changed, 136 insertions(+), 54 deletions(-) create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java index 46159a76..4d605e3c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java @@ -20,8 +20,6 @@ import static java.util.stream.Collectors.toList; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.data.Offset; import java.time.Duration; @@ -127,12 +125,6 @@ protected void consumeAvailable(Duration pollDuration, Function { - Long txId = extractTxId(record.value()); - if (txId != null) { - highestTxIdPerPartition.merge(record.partition(), txId, Math::max); - } else { - LOG.warn("txId is null for record: {}", record.key()); - } // the consumer stores the offsets that corresponds to the next record to consume, // so increment the record offset by one controlTopicOffsets.put(record.partition(), record.offset() + 1); @@ -155,7 +147,7 @@ protected Map controlTopicOffsets() { return controlTopicOffsets; } - protected Map highestTxIdPerPartition() { + protected Map highestTxIdPerPartition() { return highestTxIdPerPartition; } @@ -178,22 +170,4 @@ public void stop() { consumer.close(); admin.close(); } - - public Long extractTxId(byte[] data) { - ObjectMapper objectMapper = new ObjectMapper(); - try { - String jsonPayload = objectMapper.writeValueAsString(data); - JsonNode rootNode = objectMapper.readTree(jsonPayload); - JsonNode cdcNode = rootNode.path("_cdc"); - if (!cdcNode.isMissingNode()) { - JsonNode txIdNode = cdcNode.path("txId"); - if (!txIdNode.isMissingNode()) { - return txIdNode.asLong(); - } - } - } catch (Exception e) { - LOG.warn("Failed to extract txId from payload", e); - } - return null; - } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java index 26fca4db..178bc3b1 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java @@ -29,11 +29,13 @@ class Committable { private final ImmutableMap offsetsByTopicPartition; + private final ImmutableMap txIdsByTopicPartition; private final ImmutableList writerResults; Committable( - Map offsetsByTopicPartition, List writerResults) { + Map offsetsByTopicPartition, Map txIdsByTopicPartition, List writerResults) { this.offsetsByTopicPartition = ImmutableMap.copyOf(offsetsByTopicPartition); + this.txIdsByTopicPartition = ImmutableMap.copyOf(txIdsByTopicPartition); this.writerResults = ImmutableList.copyOf(writerResults); } @@ -41,6 +43,10 @@ public Map offsetsByTopicPartition() { return offsetsByTopicPartition; } + public Map txIdsByTopicPartition() { + return txIdsByTopicPartition; + } + public List writerResults() { return writerResults; } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java index 10730d99..0f0ec61c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java @@ -30,6 +30,8 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.ExecutionException; + +import io.tabular.iceberg.connect.data.TransactionEvent; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; @@ -105,7 +107,7 @@ private CommitterImpl( receive( envelope, // CommittableSupplier that always returns empty committables - () -> new Committable(ImmutableMap.of(), ImmutableList.of()))); + () -> new Committable(ImmutableMap.of(), ImmutableMap.of(), ImmutableList.of()))); } private Map fetchStableConsumerOffsets(String groupId) { @@ -177,9 +179,10 @@ private void sendCommitResponse(UUID commitId, CommittableSupplier committableSu .collect(toList()); Event commitReady = - new Event( + new TransactionEvent( config.controlGroupId(), - new DataComplete(commitId, assignments)); + new DataComplete(commitId, assignments), + committable.txIdsByTopicPartition()); events.add(commitReady); Map offsets = committable.offsetsByTopicPartition(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 84dd8fd8..6c07580c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -23,17 +23,19 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.data.Utilities; import java.io.IOException; import java.io.UncheckedIOException; import java.time.Duration; import java.time.OffsetDateTime; import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; + +import io.tabular.iceberg.connect.data.TransactionEvent; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -54,6 +56,7 @@ import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -120,6 +123,11 @@ private boolean receive(Envelope envelope) { return true; case DATA_COMPLETE: commitState.addReady(envelope); + if (envelope.event().payload() instanceof TransactionEvent) { + TransactionEvent event = (TransactionEvent) envelope.event(); + Map txIdPerPartition = event.txIdPerPartition(); + txIdPerPartition.forEach((k, v) -> highestTxIdPerPartition().put(k.partition(), Math.max(v, txIdPerPartition.get(k)))); + } if (commitState.isCommitReady(totalPartitionCount)) { commit(false); } @@ -140,7 +148,6 @@ private void commit(boolean partialCommit) { } private void doCommit(boolean partialCommit) { - String txidValidUntilStr = Long.toString(calculateTxidValidUntil()); Map> commitMap = commitState.tableCommitMap(); @@ -150,7 +157,7 @@ private void doCommit(boolean partialCommit) { Tasks.foreach(commitMap.entrySet()) .executeWith(exec) .stopOnFailure() - .run(entry -> commitToTable(entry.getKey(), entry.getValue(), offsetsJson, vtts, txidValidUntilStr)); + .run(entry -> commitToTable(entry.getKey(), entry.getValue(), offsetsJson, vtts)); // we should only get here if all tables committed successfully... commitConsumerOffsets(); commitState.clearResponses(); @@ -178,8 +185,7 @@ private void commitToTable( TableIdentifier tableIdentifier, List envelopeList, String offsetsJson, - OffsetDateTime vtts, - String txidValidUntilStr) { + OffsetDateTime vtts) { Table table; try { table = catalog.loadTable(tableIdentifier); @@ -217,6 +223,7 @@ private void commitToTable( if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); } else { + long txIdValidUntil = Utilities.calculateTxIdValidUntil(highestTxIdPerPartition()); if (deleteFiles.isEmpty()) { Transaction transaction = table.newTransaction(); @@ -237,7 +244,7 @@ private void commitToTable( if (vtts != null) { appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - appendOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); + appendOp.set(TXID_VALID_UNTIL_PROP, Long.toString(txIdValidUntil)); } appendOp.commit(); @@ -252,7 +259,7 @@ private void commitToTable( if (vtts != null) { deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - deltaOp.set(TXID_VALID_UNTIL_PROP, txidValidUntilStr); + deltaOp.set(TXID_VALID_UNTIL_PROP, Long.toString(txIdValidUntil)); dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); deltaOp.commit(); @@ -278,20 +285,6 @@ private void commitToTable( } } - private long calculateTxidValidUntil() { - if (highestTxIdPerPartition().isEmpty()) { - return 0L; - } - - // Find the minimum value in the map, as it represents the highest transaction ID - // that is guaranteed to be completed across all partitions - long minValue = Collections.min(highestTxIdPerPartition().values()); - - // Subtract 1 from the minimum value to get the last guaranteed completed transaction ID - // If minValue is 1, then there are no completed transactions, so return 0 - return minValue > 1 ? minValue - 1 : 0; - } - private Snapshot latestSnapshot(Table table, String branch) { if (branch == null) { return table.currentSnapshot(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index dcf57abb..89729334 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -43,10 +43,12 @@ class Worker implements Writer, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Worker.class); + private static final String COL_TXID = "_cdc_txid"; private final IcebergSinkConfig config; private final IcebergWriterFactory writerFactory; private final Map writers; private final Map sourceOffsets; + private final Map sourceTxIds; Worker(IcebergSinkConfig config, Catalog catalog) { this(config, new IcebergWriterFactory(catalog, config)); @@ -58,6 +60,7 @@ class Worker implements Writer, AutoCloseable { this.writerFactory = writerFactory; this.writers = Maps.newHashMap(); this.sourceOffsets = Maps.newHashMap(); + this.sourceTxIds = Maps.newHashMap(); } @Override @@ -65,11 +68,13 @@ public Committable committable() { List writeResults = writers.values().stream().flatMap(writer -> writer.complete().stream()).collect(toList()); Map offsets = Maps.newHashMap(sourceOffsets); + Map txIds = Maps.newHashMap(sourceTxIds); writers.clear(); sourceOffsets.clear(); + sourceTxIds.clear(); - return new Committable(offsets, writeResults); + return new Committable(offsets, txIds, writeResults); } @Override @@ -77,6 +82,7 @@ public void close() throws IOException { writers.values().forEach(RecordWriter::close); writers.clear(); sourceOffsets.clear(); + sourceTxIds.clear(); } @Override @@ -93,6 +99,10 @@ private void save(SinkRecord record) { new TopicPartition(record.topic(), record.kafkaPartition()), new Offset(record.kafkaOffset() + 1, record.timestamp())); + sourceTxIds.put( + new TopicPartition(record.topic(), record.kafkaPartition()), + Utilities.extractTxIdFromRecordValue(record.value(), COL_TXID)); + if (config.dynamicTablesEnabled()) { routeRecordDynamically(record); } else { diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java new file mode 100644 index 00000000..defc85b8 --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import org.apache.avro.Schema; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Payload; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +public class TransactionEvent extends Event { + + private Map txIdPerPartition = Maps.newHashMap(); + + public TransactionEvent(Schema schema) { + super(schema); + } + + public TransactionEvent(String groupId, Payload payload) { + super(groupId, payload); + } + + public TransactionEvent(String groupId, Payload payload, Map txIdPerPartition) { + super(groupId, payload); + this.txIdPerPartition = txIdPerPartition; + } + + public Map txIdPerPartition() { + return txIdPerPartition; + } +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index 01aebae8..b4411220 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -31,6 +31,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -132,6 +133,15 @@ public static Object extractFromRecordValue(Object recordValue, String fieldName } } + public static Long extractTxIdFromRecordValue(Object recordValue, String fieldName) { + Object txId = extractFromRecordValue(recordValue, fieldName); + if (txId instanceof Number) { + return ((Number) txId).longValue(); + } else { + return -1L; + } + } + private static Object getValueFromStruct(Struct struct, String[] fields, int idx) { Preconditions.checkArgument(idx < fields.length, "Invalid field index"); Object value = struct.get(fields[idx]); @@ -279,5 +289,19 @@ public static void close(C closeable) { } } + public static Long calculateTxIdValidUntil(Map highestTxIdPerPartition) { + if (highestTxIdPerPartition.isEmpty()) { + return 0L; + } + + // Find the minimum value in the map, as it represents the highest transaction ID + // that is guaranteed to be completed across all partitions + long minValue = Collections.min(highestTxIdPerPartition.values()); + + // Subtract 1 from the minimum value to get the last guaranteed completed transaction ID + // If minValue is 1, then there are no completed transactions, so return 0 + return minValue > 1 ? minValue - 1 : 0; + } + private Utilities() {} } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java index b3365be3..3efde8a8 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java @@ -428,6 +428,7 @@ public void testCommitShouldRespondToCommitRequest() throws IOException { () -> new Committable( sourceOffsets, + ImmutableMap.of(), ImmutableList.of( new WriterResult(TABLE_1_IDENTIFIER, dataFiles, deleteFiles, partitionStruct))); @@ -487,7 +488,7 @@ public void testCommitWhenCommittableIsEmpty() throws IOException { CONFIG.controlGroupId(), ImmutableMap.of(SOURCE_TP0, 110L, SOURCE_TP1, 100L))); CommittableSupplier committableSupplier = - () -> new Committable(ImmutableMap.of(), ImmutableList.of()); + () -> new Committable(ImmutableMap.of(), ImmutableMap.of(), ImmutableList.of()); try (CommitterImpl committerImpl = new CommitterImpl(mockContext, CONFIG, kafkaClientFactory, coordinatorThreadFactory)) { @@ -545,6 +546,7 @@ public void testCommitShouldCommitOffsetsOnlyForPartitionsWeMadeProgressOn() thr () -> new Committable( ImmutableMap.of(sourceTp1, new Offset(100L, 200L)), + ImmutableMap.of(), ImmutableList.of( new WriterResult(TABLE_1_IDENTIFIER, dataFiles, deleteFiles, partitionStruct))); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 8ce6c0c8..52c2e156 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -41,6 +41,7 @@ public class WorkerTest { private static final String SRC_TOPIC_NAME = "src-topic"; private static final String TABLE_NAME = "db.tbl"; private static final String FIELD_NAME = "fld"; + private static final String TRANSACTION_FIELD_NAME = "_cdc_txid"; @Test public void testStaticRoute() { @@ -62,6 +63,26 @@ public void testDynamicRoute() { workerTest(config, value); } + @Test + public void testStaticRouteTransactionEvent() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_NAME)); + when(config.catalogName()).thenReturn("catalog"); + Map value = ImmutableMap.of(TRANSACTION_FIELD_NAME, 743); + workerTest(config, value); + } + + @Test + public void testDynamicRouteTransactionEvent() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.dynamicTablesEnabled()).thenReturn(true); + when(config.tablesRouteField()).thenReturn(FIELD_NAME); + when(config.catalogName()).thenReturn("catalog"); + + Map value = ImmutableMap.of(TRANSACTION_FIELD_NAME, TABLE_NAME); + workerTest(config, value); + } + private void workerTest(IcebergSinkConfig config, Map value) { WriterResult writeResult = new WriterResult( From 8d47dafc72c9064722a59970f51adcce28300d0c Mon Sep 17 00:00:00 2001 From: dtobon Date: Thu, 25 Jul 2024 08:21:44 -0400 Subject: [PATCH 06/29] Add transaction consistency --- .../iceberg/connect/channel/Coordinator.java | 14 ++++++++++---- .../io/tabular/iceberg/connect/data/Utilities.java | 4 ++++ 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 6c07580c..6a0b5185 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -69,6 +69,7 @@ public class Coordinator extends Channel implements AutoCloseable { private static final String VTTS_SNAPSHOT_PROP = "kafka.connect.vtts"; private static final Duration POLL_DURATION = Duration.ofMillis(1000); private static final String TXID_VALID_UNTIL_PROP = "txid_valid_until"; + private static final String TXID_MAX_PROP = "txid_max"; private final Catalog catalog; private final IcebergSinkConfig config; @@ -126,7 +127,9 @@ private boolean receive(Envelope envelope) { if (envelope.event().payload() instanceof TransactionEvent) { TransactionEvent event = (TransactionEvent) envelope.event(); Map txIdPerPartition = event.txIdPerPartition(); - txIdPerPartition.forEach((k, v) -> highestTxIdPerPartition().put(k.partition(), Math.max(v, txIdPerPartition.get(k)))); + txIdPerPartition.forEach((k, v) -> highestTxIdPerPartition().put(k.partition(), + Math.max(v, Optional.ofNullable(highestTxIdPerPartition().get(k.partition())) + .orElse(0L)))); } if (commitState.isCommitReady(totalPartitionCount)) { commit(false); @@ -223,7 +226,8 @@ private void commitToTable( if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); } else { - long txIdValidUntil = Utilities.calculateTxIdValidUntil(highestTxIdPerPartition()); + String txIdValidUntil = Long.toString(Utilities.calculateTxIdValidUntil(highestTxIdPerPartition())); + String maxTxId = Long.toString(Utilities.getMaxTxId(highestTxIdPerPartition())); if (deleteFiles.isEmpty()) { Transaction transaction = table.newTransaction(); @@ -244,7 +248,8 @@ private void commitToTable( if (vtts != null) { appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - appendOp.set(TXID_VALID_UNTIL_PROP, Long.toString(txIdValidUntil)); + appendOp.set(TXID_VALID_UNTIL_PROP, txIdValidUntil); + appendOp.set(TXID_MAX_PROP, maxTxId); } appendOp.commit(); @@ -259,7 +264,8 @@ private void commitToTable( if (vtts != null) { deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - deltaOp.set(TXID_VALID_UNTIL_PROP, Long.toString(txIdValidUntil)); + deltaOp.set(TXID_VALID_UNTIL_PROP, txIdValidUntil); + deltaOp.set(TXID_MAX_PROP, maxTxId); dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); deltaOp.commit(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index b4411220..d421b5c8 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -303,5 +303,9 @@ public static Long calculateTxIdValidUntil(Map highestTxIdPerPartition) return minValue > 1 ? minValue - 1 : 0; } + public static Long getMaxTxId(Map highestTxIdPerPartition) { + return highestTxIdPerPartition.values().stream().max(Long::compareTo).orElse(0L); + } + private Utilities() {} } From bea5c744b45c4f721a1f0bb53d88028d725e6d6f Mon Sep 17 00:00:00 2001 From: dtobon Date: Thu, 25 Jul 2024 08:31:18 -0400 Subject: [PATCH 07/29] Add transaction consistency --- .../io/tabular/iceberg/connect/channel/Coordinator.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 6a0b5185..961ed70e 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -108,6 +108,7 @@ public void process() { new Event(config.controlGroupId(), new StartCommit(commitState.currentCommitId())); send(event); LOG.info("Sent workers commit trigger with commit-id={}", commitState.currentCommitId().toString()); + } consumeAvailable(POLL_DURATION, this::receive); @@ -151,7 +152,6 @@ private void commit(boolean partialCommit) { } private void doCommit(boolean partialCommit) { - Map> commitMap = commitState.tableCommitMap(); String offsetsJson = offsetsJson(); @@ -160,7 +160,11 @@ private void doCommit(boolean partialCommit) { Tasks.foreach(commitMap.entrySet()) .executeWith(exec) .stopOnFailure() - .run(entry -> commitToTable(entry.getKey(), entry.getValue(), offsetsJson, vtts)); + .run( + entry -> { + commitToTable(entry.getKey(), entry.getValue(), offsetsJson, vtts); + }); + // we should only get here if all tables committed successfully... commitConsumerOffsets(); commitState.clearResponses(); From 061994f0da1f5857c6fbc59b296958244f9bdbde Mon Sep 17 00:00:00 2001 From: dtobon Date: Thu, 25 Jul 2024 09:15:35 -0400 Subject: [PATCH 08/29] Fix event check --- .../java/io/tabular/iceberg/connect/channel/Coordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 961ed70e..e0050c17 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -125,7 +125,7 @@ private boolean receive(Envelope envelope) { return true; case DATA_COMPLETE: commitState.addReady(envelope); - if (envelope.event().payload() instanceof TransactionEvent) { + if (envelope.event() instanceof TransactionEvent) { TransactionEvent event = (TransactionEvent) envelope.event(); Map txIdPerPartition = event.txIdPerPartition(); txIdPerPartition.forEach((k, v) -> highestTxIdPerPartition().put(k.partition(), From a6aae8107ccd449492a50521e5e97a21aa308b33 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 31 Jul 2024 11:32:15 -0400 Subject: [PATCH 09/29] Move transaction data to payload --- kafka-connect-events/build.gradle | 1 + .../connect/events/CommitReadyPayload.java | 20 ++- .../events/TopicPartitionTransaction.java | 114 +++++++++++++++ .../connect/events/TopicPartitionTxId.java | 95 ++++++++++++ .../events/TransactionDataComplete.java | 135 ++++++++++++++++++ .../events/EventSerializationTest.java | 6 +- .../iceberg/connect/channel/CommitState.java | 8 +- .../iceberg/connect/channel/Committable.java | 9 +- .../connect/channel/CommitterImpl.java | 13 +- .../iceberg/connect/channel/Coordinator.java | 24 ++-- .../iceberg/connect/channel/EventDecoder.java | 14 +- .../connect/data/TransactionEvent.java | 49 ------- .../iceberg/connect/data/Utilities.java | 6 +- .../connect/channel/ChannelTestBase.java | 2 + .../connect/channel/CommitStateTest.java | 15 +- .../connect/channel/CommitterImplTest.java | 34 +++-- .../connect/channel/CoordinatorTest.java | 118 +++++++++++++-- .../connect/channel/EventDecoderTest.java | 22 ++- .../iceberg/connect/channel/WorkerTest.java | 18 +-- 19 files changed, 582 insertions(+), 121 deletions(-) create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java delete mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java diff --git a/kafka-connect-events/build.gradle b/kafka-connect-events/build.gradle index 9817c844..8950b0b8 100644 --- a/kafka-connect-events/build.gradle +++ b/kafka-connect-events/build.gradle @@ -7,6 +7,7 @@ dependencies { implementation libs.iceberg.common implementation libs.iceberg.guava implementation libs.avro + implementation group: 'org.apache.iceberg', name: 'iceberg-kafka-connect-events', version: '1.5.1' testImplementation libs.junit.api testRuntimeOnly libs.junit.engine diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java index fe766ac9..13c2e477 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java @@ -27,6 +27,7 @@ public class CommitReadyPayload implements Payload { private UUID commitId; private List assignments; + private List txs; private final Schema avroSchema; private static final Schema AVRO_SCHEMA = @@ -44,6 +45,13 @@ public class CommitReadyPayload implements Payload { .array() .items(TopicPartitionOffset.AVRO_SCHEMA) .noDefault() + .name("txs") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .array() + .items(TopicPartitionTxId.AVRO_SCHEMA) + .noDefault() .endRecord(); // Used by Avro reflection to instantiate this class when reading events @@ -51,9 +59,10 @@ public CommitReadyPayload(Schema avroSchema) { this.avroSchema = avroSchema; } - public CommitReadyPayload(UUID commitId, List assignments) { + public CommitReadyPayload(UUID commitId, List assignments, List txs) { this.commitId = commitId; this.assignments = assignments; + this.txs = txs; this.avroSchema = AVRO_SCHEMA; } @@ -65,6 +74,10 @@ public List assignments() { return assignments; } + public List txs() { + return txs; + } + @Override public Schema getSchema() { return avroSchema; @@ -80,6 +93,9 @@ public void put(int i, Object v) { case 1: this.assignments = (List) v; return; + case 2: + this.txs = (List) v; + return; default: // ignore the object, it must be from a newer version of the format } @@ -92,6 +108,8 @@ public Object get(int i) { return commitId; case 1: return assignments; + case 2: + return txs; default: throw new UnsupportedOperationException("Unknown field ordinal: " + i); } diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java new file mode 100644 index 00000000..3c06081e --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.events; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; + +import java.util.List; + +public class TopicPartitionTransaction implements IndexedRecord { + + private String topic; + private Integer partition; + private Long tx; + private final Schema avroSchema; + + static final int TOPIC = 10_800; + static final int PARTITION = 10_801; + static final int TX_ID = 10_802; + + public static final Types.StructType ICEBERG_SCHEMA = + Types.StructType.of( + Types.NestedField.required(TOPIC, "topic", Types.StringType.get()), + Types.NestedField.required(PARTITION, "partition", Types.IntegerType.get()), + Types.NestedField.optional(TX_ID, "tx", Types.LongType.get())); + + private static final Schema AVRO_SCHEMA = AvroSchemaUtil.convert(ICEBERG_SCHEMA, TopicPartitionTransaction.class.getName()); + + public TopicPartitionTransaction(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public TopicPartitionTransaction(String topic, Integer partition, Long tx) { + this.topic = topic; + this.partition = partition; + this.tx = tx; + this.avroSchema = AVRO_SCHEMA; + } + + public String topic() { + return topic; + } + + public Integer partition() { + return partition; + } + + public Long txId() { + return tx; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + switch (positionToId(i, avroSchema)) { + case TOPIC: + this.topic = v == null ? null : v.toString(); + break; + case PARTITION: + this.partition = (Integer) v; + break; + case TX_ID: + this.tx = (Long) v; + break; + default: + throw new IllegalArgumentException("Unknown field index: " + i); + } + } + + @Override + public Object get(int i) { + switch (positionToId(i, avroSchema)) { + case TOPIC: + return topic; + case PARTITION: + return partition; + case TX_ID: + return tx; + default: + throw new IllegalArgumentException("Unknown field index: " + i); + } + } + + static int positionToId(int position, Schema avroSchema) { + List fields = avroSchema.getFields(); + Preconditions.checkArgument( + position >= 0 && position < fields.size(), "Invalid field position: " + position); + Object val = fields.get(position).getObjectProp(AvroSchemaUtil.FIELD_ID_PROP); + return val == null ? -1 : (int) val; + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java new file mode 100644 index 00000000..55938cb4 --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java @@ -0,0 +1,95 @@ +package io.tabular.iceberg.connect.events; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.util.Utf8; + +public class TopicPartitionTxId implements Element { + + private String topic; + private Integer partition; + private Long tx; + private final Schema avroSchema; + + public static final Schema AVRO_SCHEMA = + SchemaBuilder.builder() + .record(TopicPartitionTxId.class.getName()) + .fields() + .name("topic") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .stringType() + .noDefault() + .name("partition") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .intType() + .noDefault() + .name("tx") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .longType() + .noDefault() + .endRecord(); + + // Used by Avro reflection to instantiate this class when reading events + public TopicPartitionTxId(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public TopicPartitionTxId(String topic, int partition, Long tx) { + this.topic = topic; + this.partition = partition; + this.tx = tx; + this.avroSchema = AVRO_SCHEMA; + } + + public String topic() { + return topic; + } + + public Integer partition() { + return partition; + } + + public Long tx() { + return tx; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + switch (i) { + case 0: + this.topic = v instanceof Utf8 ? v.toString() : (String) v; + return; + case 1: + this.partition = (Integer) v; + return; + case 2: + this.tx = (Long) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return topic; + case 1: + return partition; + case 2: + return tx; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} \ No newline at end of file diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java new file mode 100644 index 00000000..34f3b55e --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java @@ -0,0 +1,135 @@ +package io.tabular.iceberg.connect.events; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; + +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; + +import static org.apache.iceberg.connect.events.PayloadType.DATA_COMPLETE; + +public class TransactionDataComplete implements org.apache.iceberg.connect.events.Payload { + + private UUID commitId; + private List assignments; + private List txs; + private final Schema avroSchema; + + static final int COMMIT_ID = 10_100; + static final int ASSIGNMENTS = 10_101; + static final int ASSIGNMENTS_ELEMENT = 10_102; + static final int TX_IDS = 10_201; + static final int TX_IDS_ELEMENT = 10_202; + + private static final Types.StructType ICEBERG_SCHEMA = + Types.StructType.of( + Types.NestedField.required(COMMIT_ID, "commit_id", Types.UUIDType.get()), + Types.NestedField.optional( + ASSIGNMENTS, + "assignments", + Types.ListType.ofRequired(ASSIGNMENTS_ELEMENT, TopicPartitionOffset.ICEBERG_SCHEMA)), + Types.NestedField.optional( + TX_IDS, + "txs", + Types.ListType.ofRequired(TX_IDS_ELEMENT, TopicPartitionTransaction.ICEBERG_SCHEMA))); + + private static final Schema AVRO_SCHEMA = AvroSchemaUtil.convert(ICEBERG_SCHEMA, + TransactionDataComplete.class.getName()); + + public TransactionDataComplete(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public TransactionDataComplete(UUID commitId, List assignments, List txs) { + this.commitId = commitId; + this.assignments = assignments; + this.txs = txs; + this.avroSchema = AVRO_SCHEMA; + } + + public UUID commitId() { + return commitId; + } + + public List assignments() { + return assignments; + } + + public List txs() { + return txs; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + public PayloadType type() { + return DATA_COMPLETE; + } + + @Override + public Types.StructType writeSchema() { + return ICEBERG_SCHEMA; + } + + + @Override + @SuppressWarnings("unchecked") + public void put(int i, Object v) { + switch (positionToId(i, avroSchema)) { + case COMMIT_ID: + this.commitId = (UUID) v; + return; + case ASSIGNMENTS: + this.assignments = (List) v; + return; + case TX_IDS: + if (v instanceof List) { + List records = (List) v; + this.txs = records.stream() + .map(TransactionDataComplete::toTopicPartitionTransaction) + .collect(Collectors.toList()); + } + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (positionToId(i, avroSchema)) { + case COMMIT_ID: + return commitId; + case ASSIGNMENTS: + return assignments; + case TX_IDS: + return txs; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + static int positionToId(int position, Schema avroSchema) { + List fields = avroSchema.getFields(); + Preconditions.checkArgument( + position >= 0 && position < fields.size(), "Invalid field position: " + position); + Object val = fields.get(position).getObjectProp(AvroSchemaUtil.FIELD_ID_PROP); + return val == null ? -1 : (int) val; + } + + public static TopicPartitionTransaction toTopicPartitionTransaction(GenericData.Record record) { + Long txId = (long) record.get("tx"); + String topic = record.get("topic").toString(); + int partition = (int) record.get("partition"); + return new TopicPartitionTransaction(topic, partition, txId); + } +} diff --git a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java index e84e3c15..9e87dc0a 100644 --- a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java +++ b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.UUID; + import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types.StructType; import org.junit.jupiter.api.Test; @@ -81,7 +82,10 @@ public void testCommitReadySerialization() { commitId, Arrays.asList( new TopicPartitionOffset("topic", 1, 1L, 1L), - new TopicPartitionOffset("topic", 2, null, null)))); + new TopicPartitionOffset("topic", 2, null, null)), + Arrays.asList( + new TopicPartitionTxId("topic", 1, 1L), + new TopicPartitionTxId("topic", 2, null)))); byte[] data = Event.encode(event); Event result = Event.decode(data); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java index dfc87a57..78f454e5 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java @@ -29,6 +29,8 @@ import java.util.NoSuchElementException; import java.util.Optional; import java.util.UUID; + +import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; @@ -40,7 +42,7 @@ public class CommitState { private static final Logger LOG = LoggerFactory.getLogger(CommitState.class); private final List commitBuffer = new LinkedList<>(); - private final List readyBuffer = new LinkedList<>(); + private final List readyBuffer = new LinkedList<>(); private long startTime; private UUID currentCommitId; private final IcebergSinkConfig config; @@ -62,11 +64,11 @@ public void addResponse(Envelope envelope) { } public void addReady(Envelope envelope) { - readyBuffer.add((DataComplete) envelope.event().payload()); + readyBuffer.add((TransactionDataComplete) envelope.event().payload()); if (!isCommitInProgress()) { LOG.debug( "Received data complete for commit-id={} when no commit in progress, this can happen during recovery", - ((DataComplete) envelope.event().payload()).commitId()); + ((TransactionDataComplete) envelope.event().payload()).commitId()); } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java index 178bc3b1..83a278ef 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Committable.java @@ -25,6 +25,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.kafka.common.TopicPartition; +import java.util.stream.Collectors; class Committable { @@ -33,9 +34,13 @@ class Committable { private final ImmutableList writerResults; Committable( - Map offsetsByTopicPartition, Map txIdsByTopicPartition, List writerResults) { + Map offsetsByTopicPartition, Map txIdsByTopicPartition, List writerResults) { this.offsetsByTopicPartition = ImmutableMap.copyOf(offsetsByTopicPartition); - this.txIdsByTopicPartition = ImmutableMap.copyOf(txIdsByTopicPartition); + this.txIdsByTopicPartition = ImmutableMap.copyOf( + txIdsByTopicPartition.entrySet().stream() + .filter(entry -> entry.getValue() != null) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)) + ); this.writerResults = ImmutableList.copyOf(writerResults); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java index 0f0ec61c..0fcba3e2 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java @@ -31,9 +31,9 @@ import java.util.UUID; import java.util.concurrent.ExecutionException; -import io.tabular.iceberg.connect.data.TransactionEvent; +import io.tabular.iceberg.connect.events.TopicPartitionTransaction; +import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.PayloadType; @@ -178,11 +178,14 @@ private void sendCommitResponse(UUID commitId, CommittableSupplier committableSu }) .collect(toList()); + List txIds = committable.txIdsByTopicPartition().entrySet().stream() + .map(entry -> new TopicPartitionTransaction(entry.getKey().topic(), entry.getKey().partition(), entry.getValue())) + .collect(toList()); + Event commitReady = - new TransactionEvent( + new Event( config.controlGroupId(), - new DataComplete(commitId, assignments), - committable.txIdsByTopicPartition()); + new TransactionDataComplete(commitId, assignments, txIds)); events.add(commitReady); Map offsets = committable.offsetsByTopicPartition(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index e0050c17..b3dc3952 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -35,7 +35,8 @@ import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; -import io.tabular.iceberg.connect.data.TransactionEvent; +import io.tabular.iceberg.connect.events.TopicPartitionTransaction; +import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -56,7 +57,6 @@ import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.kafka.clients.admin.MemberDescription; -import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +68,7 @@ public class Coordinator extends Channel implements AutoCloseable { private static final String COMMIT_ID_SNAPSHOT_PROP = "kafka.connect.commit-id"; private static final String VTTS_SNAPSHOT_PROP = "kafka.connect.vtts"; private static final Duration POLL_DURATION = Duration.ofMillis(1000); - private static final String TXID_VALID_UNTIL_PROP = "txid_valid_until"; + private static final String TXID_VALID_THROUGH_PROP = "txid_valid_through"; private static final String TXID_MAX_PROP = "txid_max"; private final Catalog catalog; @@ -125,12 +125,12 @@ private boolean receive(Envelope envelope) { return true; case DATA_COMPLETE: commitState.addReady(envelope); - if (envelope.event() instanceof TransactionEvent) { - TransactionEvent event = (TransactionEvent) envelope.event(); - Map txIdPerPartition = event.txIdPerPartition(); - txIdPerPartition.forEach((k, v) -> highestTxIdPerPartition().put(k.partition(), - Math.max(v, Optional.ofNullable(highestTxIdPerPartition().get(k.partition())) - .orElse(0L)))); + if (envelope.event().payload() instanceof TransactionDataComplete) { + TransactionDataComplete payload = (TransactionDataComplete) envelope.event().payload(); + List txIds = payload.txs(); + txIds.forEach( + txId -> highestTxIdPerPartition().put(txId.partition(), + Math.max(highestTxIdPerPartition().getOrDefault(txId.partition(), 0L), txId.txId()))); } if (commitState.isCommitReady(totalPartitionCount)) { commit(false); @@ -230,7 +230,7 @@ private void commitToTable( if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); } else { - String txIdValidUntil = Long.toString(Utilities.calculateTxIdValidUntil(highestTxIdPerPartition())); + String txIdValidThrough = Long.toString(Utilities.calculateTxIdValidThrough(highestTxIdPerPartition())); String maxTxId = Long.toString(Utilities.getMaxTxId(highestTxIdPerPartition())); if (deleteFiles.isEmpty()) { Transaction transaction = table.newTransaction(); @@ -252,7 +252,7 @@ private void commitToTable( if (vtts != null) { appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - appendOp.set(TXID_VALID_UNTIL_PROP, txIdValidUntil); + appendOp.set(TXID_VALID_THROUGH_PROP, txIdValidThrough); appendOp.set(TXID_MAX_PROP, maxTxId); } @@ -268,7 +268,7 @@ private void commitToTable( if (vtts != null) { deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - deltaOp.set(TXID_VALID_UNTIL_PROP, txIdValidUntil); + deltaOp.set(TXID_VALID_THROUGH_PROP, txIdValidThrough); deltaOp.set(TXID_MAX_PROP, maxTxId); dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index 908218e5..9f8c9ba3 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -23,20 +23,24 @@ import io.tabular.iceberg.connect.events.CommitRequestPayload; import io.tabular.iceberg.connect.events.CommitResponsePayload; import io.tabular.iceberg.connect.events.CommitTablePayload; + import java.time.Instant; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.List; import java.util.stream.Collectors; + +import io.tabular.iceberg.connect.events.TopicPartitionTransaction; +import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.avro.Schema; import org.apache.avro.SchemaParseException; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.avro.AvroSchemaUtil; +import io.tabular.iceberg.connect.events.TopicPartitionTxId; import org.apache.iceberg.connect.events.AvroUtil; import org.apache.iceberg.connect.events.CommitComplete; import org.apache.iceberg.connect.events.CommitToTable; -import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.Payload; @@ -109,7 +113,13 @@ private Payload convertPayload(io.tabular.iceberg.connect.events.Payload payload : OffsetDateTime.ofInstant( Instant.ofEpochMilli(t.timestamp()), ZoneOffset.UTC))) .collect(Collectors.toList()); - return new DataComplete(pay.commitId(), converted); + List legacyTPT = pay.txs(); + List convertedTxIds = + legacyTPT.stream() + .map( + t -> new TopicPartitionTransaction(t.topic(), t.partition(), t.tx())) + .collect(Collectors.toList()); + return new TransactionDataComplete(pay.commitId(), converted, convertedTxIds); } else if (payload instanceof CommitTablePayload) { CommitTablePayload pay = (CommitTablePayload) payload; return new CommitToTable( diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java deleted file mode 100644 index defc85b8..00000000 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/TransactionEvent.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.tabular.iceberg.connect.data; - -import org.apache.avro.Schema; -import org.apache.iceberg.connect.events.Event; -import org.apache.iceberg.connect.events.Payload; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.kafka.common.TopicPartition; - -import java.util.Map; - -public class TransactionEvent extends Event { - - private Map txIdPerPartition = Maps.newHashMap(); - - public TransactionEvent(Schema schema) { - super(schema); - } - - public TransactionEvent(String groupId, Payload payload) { - super(groupId, payload); - } - - public TransactionEvent(String groupId, Payload payload, Map txIdPerPartition) { - super(groupId, payload); - this.txIdPerPartition = txIdPerPartition; - } - - public Map txIdPerPartition() { - return txIdPerPartition; - } -} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index d421b5c8..c3ee1732 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -138,7 +138,7 @@ public static Long extractTxIdFromRecordValue(Object recordValue, String fieldNa if (txId instanceof Number) { return ((Number) txId).longValue(); } else { - return -1L; + return null; } } @@ -289,13 +289,13 @@ public static void close(C closeable) { } } - public static Long calculateTxIdValidUntil(Map highestTxIdPerPartition) { + public static Long calculateTxIdValidThrough(Map highestTxIdPerPartition) { if (highestTxIdPerPartition.isEmpty()) { return 0L; } // Find the minimum value in the map, as it represents the highest transaction ID - // that is guaranteed to be completed across all partitions + // that is common across all partitions long minValue = Collections.min(highestTxIdPerPartition.values()); // Subtract 1 from the minimum value to get the last guaranteed completed transaction ID diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java index 5b2bbc74..0869280e 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java @@ -86,6 +86,8 @@ private InMemoryCatalog initInMemoryCatalog() { protected static final String OFFSETS_SNAPSHOT_PROP = String.format("kafka.connect.offsets.%s.%s", CTL_TOPIC_NAME, CONTROL_CONSUMER_GROUP_ID); protected static final String VTTS_SNAPSHOT_PROP = "kafka.connect.vtts"; + protected static final String TX_ID_VALID_THROUGH_PROP = "txid_valid_through"; + protected static final String MAX_TX_ID__PROP = "txid_max"; @BeforeEach @SuppressWarnings("deprecation") diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java index 9ee165d9..be6caac4 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java @@ -27,7 +27,8 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.UUID; -import org.apache.iceberg.connect.events.DataComplete; + +import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.Payload; import org.apache.iceberg.connect.events.TopicPartitionOffset; @@ -47,15 +48,15 @@ public void testIsCommitReady() { CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); commitState.startNewCommit(); - DataComplete payload1 = mock(DataComplete.class); + TransactionDataComplete payload1 = mock(TransactionDataComplete.class); when(payload1.commitId()).thenReturn(commitState.currentCommitId()); when(payload1.assignments()).thenReturn(ImmutableList.of(tp, tp)); - DataComplete payload2 = mock(DataComplete.class); + TransactionDataComplete payload2 = mock(TransactionDataComplete.class); when(payload2.commitId()).thenReturn(commitState.currentCommitId()); when(payload2.assignments()).thenReturn(ImmutableList.of(tp)); - DataComplete payload3 = mock(DataComplete.class); + TransactionDataComplete payload3 = mock(TransactionDataComplete.class); when(payload3.commitId()).thenReturn(UUID.randomUUID()); when(payload3.assignments()).thenReturn(ImmutableList.of(tp)); @@ -69,14 +70,14 @@ public void testIsCommitReady() { @Test public void testGetVtts() { - DataComplete payload1 = mock(DataComplete.class); + TransactionDataComplete payload1 = mock(TransactionDataComplete.class); TopicPartitionOffset tp1 = mock(TopicPartitionOffset.class); when(tp1.timestamp()).thenReturn(offsetDateTime(3L)); TopicPartitionOffset tp2 = mock(TopicPartitionOffset.class); when(tp2.timestamp()).thenReturn(offsetDateTime(2L)); when(payload1.assignments()).thenReturn(ImmutableList.of(tp1, tp2)); - DataComplete payload2 = mock(DataComplete.class); + TransactionDataComplete payload2 = mock(TransactionDataComplete.class); TopicPartitionOffset tp3 = mock(TopicPartitionOffset.class); when(tp3.timestamp()).thenReturn(offsetDateTime(1L)); when(payload2.assignments()).thenReturn(ImmutableList.of(tp3)); @@ -91,7 +92,7 @@ public void testGetVtts() { assertThat(commitState.vtts(true)).isNull(); // null timestamp for one, so should not set a vtts - DataComplete payload3 = mock(DataComplete.class); + TransactionDataComplete payload3 = mock(TransactionDataComplete.class); TopicPartitionOffset tp4 = mock(TopicPartitionOffset.class); when(tp4.timestamp()).thenReturn(null); when(payload3.assignments()).thenReturn(ImmutableList.of(tp4)); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java index 3efde8a8..71efc518 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java @@ -41,6 +41,8 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; + +import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.commons.lang3.NotImplementedException; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -49,7 +51,6 @@ import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.connect.events.AvroUtil; import org.apache.iceberg.connect.events.CommitComplete; -import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.PayloadType; @@ -259,13 +260,14 @@ private void assertDataComplete( ProducerRecord producerRecord, UUID expectedProducerId, UUID expectedCommitId, - Map> expectedAssignments) { + Map> expectedAssignments, + Map expectedTxIds) { assertThat(producerRecord.key()).isEqualTo(expectedProducerId.toString()); Event event = AvroUtil.decode(producerRecord.value()); assertThat(event.type()).isEqualTo(PayloadType.DATA_COMPLETE); - assertThat(event.payload()).isInstanceOf(DataComplete.class); - DataComplete commitReadyPayload = (DataComplete) event.payload(); + assertThat(event.payload()).isInstanceOf(TransactionDataComplete.class); + TransactionDataComplete commitReadyPayload = (TransactionDataComplete) event.payload(); assertThat(commitReadyPayload.commitId()).isEqualTo(expectedCommitId); assertThat( commitReadyPayload.assignments().stream() @@ -279,6 +281,18 @@ private void assertDataComplete( expectedAssignments.entrySet().stream() .map(e -> Pair.of(e.getKey(), e.getValue())) .collect(Collectors.toList())); + + assertThat( + commitReadyPayload.txs().stream() + .map( + x -> + Pair.of( + new TopicPartition(x.topic(), x.partition()), x.txId())) + .collect(Collectors.toList())) + .isEqualTo( + expectedTxIds.entrySet().stream() + .map(e -> Pair.of(e.getKey(), e.getValue())) + .collect(Collectors.toList())); } private OffsetDateTime offsetDateTime(Long ms) { @@ -424,11 +438,12 @@ public void testCommitShouldRespondToCommitRequest() throws IOException { List deleteFiles = ImmutableList.of(); Types.StructType partitionStruct = Types.StructType.of(); Map sourceOffsets = ImmutableMap.of(SOURCE_TP0, new Offset(100L, 200L)); + Map sourceTxIds = ImmutableMap.of(SOURCE_TP0, 100L); CommittableSupplier committableSupplier = () -> new Committable( sourceOffsets, - ImmutableMap.of(), + sourceTxIds, ImmutableList.of( new WriterResult(TABLE_1_IDENTIFIER, dataFiles, deleteFiles, partitionStruct))); @@ -463,7 +478,8 @@ public void testCommitShouldRespondToCommitRequest() throws IOException { producer.history().get(1), producerId, commitId, - ImmutableMap.of(SOURCE_TP0, Pair.of(100L, offsetDateTime(200L)))); + ImmutableMap.of(SOURCE_TP0, Pair.of(100L, offsetDateTime(200L))), + ImmutableMap.of(SOURCE_TP0, 100L)); assertThat(producer.consumerGroupOffsetsHistory()).hasSize(2); Map expectedConsumerOffset = @@ -515,7 +531,8 @@ public void testCommitWhenCommittableIsEmpty() throws IOException { producer.history().get(0), producerId, commitId, - ImmutableMap.of(SOURCE_TP0, Pair.of(null, null))); + ImmutableMap.of(SOURCE_TP0, Pair.of(null, null)), + ImmutableMap.of()); assertThat(producer.consumerGroupOffsetsHistory()).hasSize(0); } @@ -583,7 +600,8 @@ public void testCommitShouldCommitOffsetsOnlyForPartitionsWeMadeProgressOn() thr commitId, ImmutableMap.of( sourceTp0, Pair.of(null, null), - sourceTp1, Pair.of(100L, offsetDateTime(200L)))); + sourceTp1, Pair.of(100L, offsetDateTime(200L))), + ImmutableMap.of()); assertThat(producer.consumerGroupOffsetsHistory()).hasSize(2); Map expectedConsumerOffset = diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java index 9cd34f1d..e8cf188b 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java @@ -21,6 +21,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.when; +import io.tabular.iceberg.connect.events.TopicPartitionTransaction; +import io.tabular.iceberg.connect.events.TransactionDataComplete; import io.tabular.iceberg.connect.fixtures.EventTestUtil; import java.time.Instant; import java.time.OffsetDateTime; @@ -43,7 +45,6 @@ import org.apache.iceberg.connect.events.AvroUtil; import org.apache.iceberg.connect.events.CommitComplete; import org.apache.iceberg.connect.events.CommitToTable; -import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.PayloadType; @@ -69,9 +70,11 @@ public class CoordinatorTest extends ChannelTestBase { public void testCommitAppend() { Assertions.assertEquals(0, ImmutableList.copyOf(table.snapshots().iterator()).size()); + List transactionsProcessed = + ImmutableList.of(new TopicPartitionTransaction("topic", 1, 100L)); OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); UUID commitId = - coordinatorTest(ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), ts); + coordinatorTest(ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), ts, transactionsProcessed); table.refresh(); assertThat(producer.history()).hasSize(3); @@ -93,16 +96,27 @@ public void testCommitAppend() { Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); Assertions.assertEquals( Long.toString(ts.toInstant().toEpochMilli()), summary.get(VTTS_SNAPSHOT_PROP)); + Assertions.assertEquals(transactionsProcessed.get(0).txId() - 1, Long.valueOf(summary.get(TX_ID_VALID_THROUGH_PROP))); + Assertions.assertEquals(transactionsProcessed.get(0).txId(), Long.valueOf(summary.get(MAX_TX_ID__PROP))); } @Test public void testCommitDelta() { + List transactionsProcessed = + ImmutableList.of(new TopicPartitionTransaction("topic", 1, 100L), + new TopicPartitionTransaction("topic", 2, 102L), + new TopicPartitionTransaction("topic", 3, 101L), + new TopicPartitionTransaction("topic", 3, 102L), + new TopicPartitionTransaction("topic", 3, 100L), + new TopicPartitionTransaction("topic", 3, 103L), + new TopicPartitionTransaction("topic", 4, 104L)); OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); UUID commitId = coordinatorTest( ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(EventTestUtil.createDeleteFile()), - ts); + ts, + transactionsProcessed); assertThat(producer.history()).hasSize(3); assertThat(consumer.committed(ImmutableSet.of(CTL_TOPIC_PARTITION))) @@ -123,12 +137,14 @@ public void testCommitDelta() { Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); Assertions.assertEquals( Long.toString(ts.toInstant().toEpochMilli()), summary.get(VTTS_SNAPSHOT_PROP)); + Assertions.assertEquals(99L, Long.valueOf(summary.get(TX_ID_VALID_THROUGH_PROP))); + Assertions.assertEquals(104L, Long.valueOf(summary.get(MAX_TX_ID__PROP))); } @Test public void testCommitNoFiles() { OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); - UUID commitId = coordinatorTest(ImmutableList.of(), ImmutableList.of(), ts); + UUID commitId = coordinatorTest(ImmutableList.of(), ImmutableList.of(), ts, ImmutableList.of()); assertThat(producer.history()).hasSize(2); assertThat(consumer.committed(ImmutableSet.of(CTL_TOPIC_PARTITION))) @@ -155,7 +171,8 @@ public void testCommitError() { coordinatorTest( ImmutableList.of(badDataFile), ImmutableList.of(), - OffsetDateTime.ofInstant(Instant.ofEpochMilli(0L), ZoneOffset.UTC)); + OffsetDateTime.ofInstant(Instant.ofEpochMilli(0L), ZoneOffset.UTC), + ImmutableList.of()); // no commit messages sent assertThat(producer.history()).hasSize(1); @@ -168,6 +185,8 @@ public void testCommitError() { @Test public void testShouldDeduplicateDataFilesBeforeAppending() { + List transactionsProcessed = + ImmutableList.of(new TopicPartitionTransaction("topic", 1, 100L)); OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); DataFile dataFile = EventTestUtil.createDataFile(); @@ -189,9 +208,10 @@ public void testShouldDeduplicateDataFilesBeforeAppending() { commitResponse, // duplicate commit response new Event( config.controlGroupId(), - new DataComplete( + new TransactionDataComplete( currentCommitId, - ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts))))); + ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)), + transactionsProcessed))); }); assertCommitTable(1, commitId, ts); @@ -204,6 +224,9 @@ public void testShouldDeduplicateDataFilesBeforeAppending() { Assertions.assertEquals(DataOperations.APPEND, snapshot.operation()); Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); Assertions.assertEquals(0, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); + Map summary = snapshot.summary(); + Assertions.assertEquals(transactionsProcessed.get(0).txId() - 1, Long.valueOf(summary.get(TX_ID_VALID_THROUGH_PROP))); + Assertions.assertEquals(transactionsProcessed.get(0).txId(), Long.valueOf(summary.get(MAX_TX_ID__PROP))); } @Test @@ -229,9 +252,10 @@ public void testShouldDeduplicateDeleteFilesBeforeAppending() { duplicateCommitResponse, // duplicate commit response new Event( config.controlGroupId(), - new DataComplete( - currentCommitId, - ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts))))); + new TransactionDataComplete( + currentCommitId, + ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)), + ImmutableList.of(new TopicPartitionTransaction("topic", 1, 100L))))); }); assertCommitTable(1, commitId, ts); @@ -263,6 +287,39 @@ private void validateAddedFiles( .collect(Collectors.toSet())); } + @Test + public void testTxIdValidThroughInSnapshotSummary() { + Assertions.assertEquals(0, ImmutableList.copyOf(table.snapshots().iterator()).size()); + + Map txIdPerPartition = ImmutableMap.of( + new TopicPartition("topic", 1), 100L, + new TopicPartition("topic", 2), 102L); + OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); + UUID commitId = + coordinatorTxIdValidThroughTest(ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), ts, txIdPerPartition); + table.refresh(); + + assertThat(producer.history()).hasSize(3); + assertThat(consumer.committed(ImmutableSet.of(CTL_TOPIC_PARTITION))) + .isEqualTo(ImmutableMap.of(CTL_TOPIC_PARTITION, new OffsetAndMetadata(3L))); + assertCommitTable(1, commitId, ts); + assertCommitComplete(2, commitId, ts); + + List snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(1, snapshots.size()); + + Snapshot snapshot = snapshots.get(0); + Assertions.assertEquals(DataOperations.APPEND, snapshot.operation()); + Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); + Assertions.assertEquals(0, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); + + Map summary = snapshot.summary(); + Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); + Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); + Assertions.assertEquals( + Long.toString(ts.toInstant().toEpochMilli()), summary.get(VTTS_SNAPSHOT_PROP)); + } + /** * * @@ -356,7 +413,7 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { AvroUtil.encode( new Event( config.controlGroupId(), - new DataComplete( + new TransactionDataComplete( commitId, ImmutableList.of( new TopicPartitionOffset( @@ -364,7 +421,9 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { 0, 100L, OffsetDateTime.ofInstant( - Instant.ofEpochMilli(100L), ZoneOffset.UTC)))))))); + Instant.ofEpochMilli(100L), ZoneOffset.UTC))), + ImmutableList.of( + new TopicPartitionTransaction(SRC_TOPIC_NAME, 0, 100L))))))); currentControlTopicOffset += 1; } @@ -428,7 +487,7 @@ private void assertCommitComplete(int idx, UUID commitId, OffsetDateTime ts) { } private UUID coordinatorTest( - List dataFiles, List deleteFiles, OffsetDateTime ts) { + List dataFiles, List deleteFiles, OffsetDateTime ts, List transactionsProcessed) { return coordinatorTest( currentCommitId -> { Event commitResponse = @@ -444,14 +503,43 @@ private UUID coordinatorTest( Event commitReady = new Event( config.controlGroupId(), - new DataComplete( + new TransactionDataComplete( currentCommitId, - ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)))); + ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)), + transactionsProcessed)); return ImmutableList.of(commitResponse, commitReady); }); } + private UUID coordinatorTxIdValidThroughTest( + List dataFiles, List deleteFiles, OffsetDateTime ts, Map txIdPerPartition) { + return coordinatorTest( + currentCommitId -> { + Event commitResponse = + new Event( + config.controlGroupId(), + new DataWritten( + StructType.of(), + currentCommitId, + new TableReference("catalog", ImmutableList.of("db"), "tbl"), + dataFiles, + deleteFiles)); + + Event commitReady = + new Event( + config.controlGroupId(), + new TransactionDataComplete( + currentCommitId, + ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)), + txIdPerPartition.entrySet().stream() + .map(entry -> new TopicPartitionTransaction(entry.getKey().topic(), entry.getKey().partition(), entry.getValue())) + .collect(Collectors.toList()))); + + return ImmutableList.of(commitResponse, commitReady); + }); + } + private UUID coordinatorTest(Function> eventsFn) { when(config.commitIntervalMs()).thenReturn(0); when(config.commitTimeoutMs()).thenReturn(Integer.MAX_VALUE); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java index 54e157de..5e8f7f8c 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java @@ -37,13 +37,15 @@ import java.util.Collections; import java.util.List; import java.util.UUID; + +import io.tabular.iceberg.connect.events.TopicPartitionTxId; +import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.avro.Schema; import org.apache.iceberg.DataFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.events.CommitComplete; import org.apache.iceberg.connect.events.CommitToTable; -import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.PayloadType; @@ -190,7 +192,11 @@ public void testCommitReadyBecomesDataComplete() { commitId, Arrays.asList( new TopicPartitionOffset("topic", 1, 1L, 1L), - new TopicPartitionOffset("topic", 2, null, null)))); + new TopicPartitionOffset("topic", 2, null, null)), + Arrays.asList( + new TopicPartitionTxId("topic", 1, 1L), + new TopicPartitionTxId("topic", 2, null)))); + byte[] data = io.tabular.iceberg.connect.events.Event.encode(event); @@ -198,8 +204,8 @@ public void testCommitReadyBecomesDataComplete() { assertThat(event.groupId()).isEqualTo("cg-connector"); assertThat(result.type()).isEqualTo(PayloadType.DATA_COMPLETE); - assertThat(result.payload()).isInstanceOf(DataComplete.class); - DataComplete payload = (DataComplete) result.payload(); + assertThat(result.payload()).isInstanceOf(TransactionDataComplete.class); + TransactionDataComplete payload = (TransactionDataComplete) result.payload(); assertThat(payload.commitId()).isEqualTo(commitId); assertThat(payload.assignments().get(0).topic()).isEqualTo("topic"); @@ -212,6 +218,14 @@ public void testCommitReadyBecomesDataComplete() { assertThat(payload.assignments().get(1).partition()).isEqualTo(2); assertThat(payload.assignments().get(1).offset()).isNull(); assertThat(payload.assignments().get(1).timestamp()).isNull(); + + assertThat(payload.txs().get(0).topic()).isEqualTo("topic"); + assertThat(payload.txs().get(0).partition()).isEqualTo(1); + assertThat(payload.txs().get(0).txId()).isEqualTo(1L); + + assertThat(payload.txs().get(1).topic()).isEqualTo("topic"); + assertThat(payload.txs().get(1).partition()).isEqualTo(2); + assertThat(payload.txs().get(1).txId()).isNull(); } @Test diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 52c2e156..420ebfad 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -85,11 +85,11 @@ public void testDynamicRouteTransactionEvent() { private void workerTest(IcebergSinkConfig config, Map value) { WriterResult writeResult = - new WriterResult( - TableIdentifier.parse(TABLE_NAME), - ImmutableList.of(EventTestUtil.createDataFile()), - ImmutableList.of(), - StructType.of()); + new WriterResult( + TableIdentifier.parse(TABLE_NAME), + ImmutableList.of(EventTestUtil.createDataFile()), + ImmutableList.of(), + StructType.of()); IcebergWriter writer = mock(IcebergWriter.class); when(writer.complete()).thenReturn(ImmutableList.of(writeResult)); @@ -108,9 +108,9 @@ private void workerTest(IcebergSinkConfig config, Map value) { // offset should be one more than the record offset assertThat( committable - .offsetsByTopicPartition() - .get(committable.offsetsByTopicPartition().keySet().iterator().next()) - .offset()) - .isEqualTo(1L); + .offsetsByTopicPartition() + .get(committable.offsetsByTopicPartition().keySet().iterator().next()) + .offset()) + .isEqualTo(1L); } } From 51566f40700c16c72478572f1d47232c4acefac6 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 31 Jul 2024 11:45:43 -0400 Subject: [PATCH 10/29] Rename fields --- .../events/TopicPartitionTransaction.java | 14 +++++++------- .../connect/events/TopicPartitionTxId.java | 16 ++++++++-------- .../events/TransactionDataComplete.java | 18 +++++++++--------- .../iceberg/connect/channel/CommitState.java | 1 - .../iceberg/connect/channel/Coordinator.java | 2 +- .../iceberg/connect/channel/EventDecoder.java | 2 +- .../connect/channel/CommitterImplTest.java | 2 +- .../connect/channel/EventDecoderTest.java | 12 ++++++------ 8 files changed, 33 insertions(+), 34 deletions(-) diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java index 3c06081e..48ec177a 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java @@ -30,7 +30,7 @@ public class TopicPartitionTransaction implements IndexedRecord { private String topic; private Integer partition; - private Long tx; + private Long txId; private final Schema avroSchema; static final int TOPIC = 10_800; @@ -41,7 +41,7 @@ public class TopicPartitionTransaction implements IndexedRecord { Types.StructType.of( Types.NestedField.required(TOPIC, "topic", Types.StringType.get()), Types.NestedField.required(PARTITION, "partition", Types.IntegerType.get()), - Types.NestedField.optional(TX_ID, "tx", Types.LongType.get())); + Types.NestedField.optional(TX_ID, "txId", Types.LongType.get())); private static final Schema AVRO_SCHEMA = AvroSchemaUtil.convert(ICEBERG_SCHEMA, TopicPartitionTransaction.class.getName()); @@ -49,10 +49,10 @@ public TopicPartitionTransaction(Schema avroSchema) { this.avroSchema = avroSchema; } - public TopicPartitionTransaction(String topic, Integer partition, Long tx) { + public TopicPartitionTransaction(String topic, Integer partition, Long txId) { this.topic = topic; this.partition = partition; - this.tx = tx; + this.txId = txId; this.avroSchema = AVRO_SCHEMA; } @@ -65,7 +65,7 @@ public Integer partition() { } public Long txId() { - return tx; + return txId; } @Override @@ -83,7 +83,7 @@ public void put(int i, Object v) { this.partition = (Integer) v; break; case TX_ID: - this.tx = (Long) v; + this.txId = (Long) v; break; default: throw new IllegalArgumentException("Unknown field index: " + i); @@ -98,7 +98,7 @@ public Object get(int i) { case PARTITION: return partition; case TX_ID: - return tx; + return txId; default: throw new IllegalArgumentException("Unknown field index: " + i); } diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java index 55938cb4..22ff33ad 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java @@ -8,7 +8,7 @@ public class TopicPartitionTxId implements Element { private String topic; private Integer partition; - private Long tx; + private Long txId; private final Schema avroSchema; public static final Schema AVRO_SCHEMA = @@ -25,7 +25,7 @@ public class TopicPartitionTxId implements Element { .type() .intType() .noDefault() - .name("tx") + .name("tx_id") .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) .type() .nullable() @@ -38,10 +38,10 @@ public TopicPartitionTxId(Schema avroSchema) { this.avroSchema = avroSchema; } - public TopicPartitionTxId(String topic, int partition, Long tx) { + public TopicPartitionTxId(String topic, int partition, Long txId) { this.topic = topic; this.partition = partition; - this.tx = tx; + this.txId = txId; this.avroSchema = AVRO_SCHEMA; } @@ -53,8 +53,8 @@ public Integer partition() { return partition; } - public Long tx() { - return tx; + public Long txId() { + return txId; } @Override @@ -72,7 +72,7 @@ public void put(int i, Object v) { this.partition = (Integer) v; return; case 2: - this.tx = (Long) v; + this.txId = (Long) v; return; default: // ignore the object, it must be from a newer version of the format @@ -87,7 +87,7 @@ public Object get(int i) { case 1: return partition; case 2: - return tx; + return txId; default: throw new UnsupportedOperationException("Unknown field ordinal: " + i); } diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java index 34f3b55e..b0023bd4 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java @@ -18,7 +18,7 @@ public class TransactionDataComplete implements org.apache.iceberg.connect.event private UUID commitId; private List assignments; - private List txs; + private List txIds; private final Schema avroSchema; static final int COMMIT_ID = 10_100; @@ -36,7 +36,7 @@ public class TransactionDataComplete implements org.apache.iceberg.connect.event Types.ListType.ofRequired(ASSIGNMENTS_ELEMENT, TopicPartitionOffset.ICEBERG_SCHEMA)), Types.NestedField.optional( TX_IDS, - "txs", + "tx_ids", Types.ListType.ofRequired(TX_IDS_ELEMENT, TopicPartitionTransaction.ICEBERG_SCHEMA))); private static final Schema AVRO_SCHEMA = AvroSchemaUtil.convert(ICEBERG_SCHEMA, @@ -46,10 +46,10 @@ public TransactionDataComplete(Schema avroSchema) { this.avroSchema = avroSchema; } - public TransactionDataComplete(UUID commitId, List assignments, List txs) { + public TransactionDataComplete(UUID commitId, List assignments, List txIds) { this.commitId = commitId; this.assignments = assignments; - this.txs = txs; + this.txIds = txIds; this.avroSchema = AVRO_SCHEMA; } @@ -61,8 +61,8 @@ public List assignments() { return assignments; } - public List txs() { - return txs; + public List txIds() { + return txIds; } @Override @@ -94,7 +94,7 @@ public void put(int i, Object v) { case TX_IDS: if (v instanceof List) { List records = (List) v; - this.txs = records.stream() + this.txIds = records.stream() .map(TransactionDataComplete::toTopicPartitionTransaction) .collect(Collectors.toList()); } @@ -112,7 +112,7 @@ public Object get(int i) { case ASSIGNMENTS: return assignments; case TX_IDS: - return txs; + return txIds; default: throw new UnsupportedOperationException("Unknown field ordinal: " + i); } @@ -127,7 +127,7 @@ static int positionToId(int position, Schema avroSchema) { } public static TopicPartitionTransaction toTopicPartitionTransaction(GenericData.Record record) { - Long txId = (long) record.get("tx"); + Long txId = (long) record.get("txId"); String topic = record.get("topic").toString(); int partition = (int) record.get("partition"); return new TopicPartitionTransaction(topic, partition, txId); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java index 78f454e5..135f3d53 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java @@ -32,7 +32,6 @@ import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.connect.events.DataComplete; import org.apache.iceberg.connect.events.DataWritten; import org.apache.iceberg.connect.events.TopicPartitionOffset; import org.slf4j.Logger; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index b3dc3952..37f9754b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -127,7 +127,7 @@ private boolean receive(Envelope envelope) { commitState.addReady(envelope); if (envelope.event().payload() instanceof TransactionDataComplete) { TransactionDataComplete payload = (TransactionDataComplete) envelope.event().payload(); - List txIds = payload.txs(); + List txIds = payload.txIds(); txIds.forEach( txId -> highestTxIdPerPartition().put(txId.partition(), Math.max(highestTxIdPerPartition().getOrDefault(txId.partition(), 0L), txId.txId()))); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index 9f8c9ba3..06cb157b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -117,7 +117,7 @@ private Payload convertPayload(io.tabular.iceberg.connect.events.Payload payload List convertedTxIds = legacyTPT.stream() .map( - t -> new TopicPartitionTransaction(t.topic(), t.partition(), t.tx())) + t -> new TopicPartitionTransaction(t.topic(), t.partition(), t.txId())) .collect(Collectors.toList()); return new TransactionDataComplete(pay.commitId(), converted, convertedTxIds); } else if (payload instanceof CommitTablePayload) { diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java index 71efc518..c36068d3 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java @@ -283,7 +283,7 @@ private void assertDataComplete( .collect(Collectors.toList())); assertThat( - commitReadyPayload.txs().stream() + commitReadyPayload.txIds().stream() .map( x -> Pair.of( diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java index 5e8f7f8c..b94df2f3 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java @@ -219,13 +219,13 @@ public void testCommitReadyBecomesDataComplete() { assertThat(payload.assignments().get(1).offset()).isNull(); assertThat(payload.assignments().get(1).timestamp()).isNull(); - assertThat(payload.txs().get(0).topic()).isEqualTo("topic"); - assertThat(payload.txs().get(0).partition()).isEqualTo(1); - assertThat(payload.txs().get(0).txId()).isEqualTo(1L); + assertThat(payload.txIds().get(0).topic()).isEqualTo("topic"); + assertThat(payload.txIds().get(0).partition()).isEqualTo(1); + assertThat(payload.txIds().get(0).txId()).isEqualTo(1L); - assertThat(payload.txs().get(1).topic()).isEqualTo("topic"); - assertThat(payload.txs().get(1).partition()).isEqualTo(2); - assertThat(payload.txs().get(1).txId()).isNull(); + assertThat(payload.txIds().get(1).topic()).isEqualTo("topic"); + assertThat(payload.txIds().get(1).partition()).isEqualTo(2); + assertThat(payload.txIds().get(1).txId()).isNull(); } @Test From 5611f6a98d951ef95fedd84479758a1a67d12de7 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 31 Jul 2024 14:42:17 -0400 Subject: [PATCH 11/29] Clean up and testing --- .../connect/events/CommitReadyPayload.java | 16 +++++++------- .../iceberg/connect/channel/Coordinator.java | 6 ++++-- .../iceberg/connect/channel/EventDecoder.java | 2 +- .../iceberg/connect/channel/Worker.java | 9 +++++--- .../connect/channel/CommitStateTest.java | 5 +++++ .../connect/channel/CoordinatorTest.java | 15 +++++++++++-- .../iceberg/connect/channel/WorkerTest.java | 21 ++++++++----------- 7 files changed, 46 insertions(+), 28 deletions(-) diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java index 13c2e477..3b435a71 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java @@ -27,7 +27,7 @@ public class CommitReadyPayload implements Payload { private UUID commitId; private List assignments; - private List txs; + private List txIds; private final Schema avroSchema; private static final Schema AVRO_SCHEMA = @@ -45,7 +45,7 @@ public class CommitReadyPayload implements Payload { .array() .items(TopicPartitionOffset.AVRO_SCHEMA) .noDefault() - .name("txs") + .name("txIds") .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) .type() .nullable() @@ -59,10 +59,10 @@ public CommitReadyPayload(Schema avroSchema) { this.avroSchema = avroSchema; } - public CommitReadyPayload(UUID commitId, List assignments, List txs) { + public CommitReadyPayload(UUID commitId, List assignments, List txIds) { this.commitId = commitId; this.assignments = assignments; - this.txs = txs; + this.txIds = txIds; this.avroSchema = AVRO_SCHEMA; } @@ -74,8 +74,8 @@ public List assignments() { return assignments; } - public List txs() { - return txs; + public List txIds() { + return txIds; } @Override @@ -94,7 +94,7 @@ public void put(int i, Object v) { this.assignments = (List) v; return; case 2: - this.txs = (List) v; + this.txIds = (List) v; return; default: // ignore the object, it must be from a newer version of the format @@ -109,7 +109,7 @@ public Object get(int i) { case 1: return assignments; case 2: - return txs; + return txIds; default: throw new UnsupportedOperationException("Unknown field ordinal: " + i); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 37f9754b..95faf2d4 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -128,9 +128,11 @@ private boolean receive(Envelope envelope) { if (envelope.event().payload() instanceof TransactionDataComplete) { TransactionDataComplete payload = (TransactionDataComplete) envelope.event().payload(); List txIds = payload.txIds(); + if (txIds != null) { txIds.forEach( - txId -> highestTxIdPerPartition().put(txId.partition(), - Math.max(highestTxIdPerPartition().getOrDefault(txId.partition(), 0L), txId.txId()))); + txId -> highestTxIdPerPartition().put(txId.partition(), + Math.max(highestTxIdPerPartition().getOrDefault(txId.partition(), 0L), txId.txId()))); + } } if (commitState.isCommitReady(totalPartitionCount)) { commit(false); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index 06cb157b..56994dbc 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -113,7 +113,7 @@ private Payload convertPayload(io.tabular.iceberg.connect.events.Payload payload : OffsetDateTime.ofInstant( Instant.ofEpochMilli(t.timestamp()), ZoneOffset.UTC))) .collect(Collectors.toList()); - List legacyTPT = pay.txs(); + List legacyTPT = pay.txIds(); List convertedTxIds = legacyTPT.stream() .map( diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 89729334..ae2ffde5 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -99,9 +99,12 @@ private void save(SinkRecord record) { new TopicPartition(record.topic(), record.kafkaPartition()), new Offset(record.kafkaOffset() + 1, record.timestamp())); - sourceTxIds.put( - new TopicPartition(record.topic(), record.kafkaPartition()), - Utilities.extractTxIdFromRecordValue(record.value(), COL_TXID)); + Long txId = Utilities.extractTxIdFromRecordValue(record.value(), COL_TXID); + if (txId != null) { + sourceTxIds.put( + new TopicPartition(record.topic(), record.kafkaPartition()), + txId); + } if (config.dynamicTablesEnabled()) { routeRecordDynamically(record); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java index be6caac4..b836df17 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java @@ -28,6 +28,7 @@ import java.time.ZoneOffset; import java.util.UUID; +import io.tabular.iceberg.connect.events.TopicPartitionTransaction; import io.tabular.iceberg.connect.events.TransactionDataComplete; import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.Payload; @@ -44,6 +45,7 @@ private OffsetDateTime offsetDateTime(Long ts) { @Test public void testIsCommitReady() { TopicPartitionOffset tp = mock(TopicPartitionOffset.class); + TopicPartitionTransaction tpt = mock(TopicPartitionTransaction.class); CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); commitState.startNewCommit(); @@ -51,14 +53,17 @@ public void testIsCommitReady() { TransactionDataComplete payload1 = mock(TransactionDataComplete.class); when(payload1.commitId()).thenReturn(commitState.currentCommitId()); when(payload1.assignments()).thenReturn(ImmutableList.of(tp, tp)); + when(payload1.txIds()).thenReturn(ImmutableList.of(tpt, tpt)); TransactionDataComplete payload2 = mock(TransactionDataComplete.class); when(payload2.commitId()).thenReturn(commitState.currentCommitId()); when(payload2.assignments()).thenReturn(ImmutableList.of(tp)); + when(payload2.txIds()).thenReturn(ImmutableList.of(tpt)); TransactionDataComplete payload3 = mock(TransactionDataComplete.class); when(payload3.commitId()).thenReturn(UUID.randomUUID()); when(payload3.assignments()).thenReturn(ImmutableList.of(tp)); + when(payload3.txIds()).thenReturn(ImmutableList.of(tpt)); commitState.addReady(wrapInEnvelope(payload1)); commitState.addReady(wrapInEnvelope(payload2)); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java index e8cf188b..8ecc3d20 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java @@ -116,7 +116,7 @@ public void testCommitDelta() { ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(EventTestUtil.createDeleteFile()), ts, - transactionsProcessed); + transactionsProcessed); assertThat(producer.history()).hasSize(3); assertThat(consumer.committed(ImmutableSet.of(CTL_TOPIC_PARTITION))) @@ -318,6 +318,8 @@ public void testTxIdValidThroughInSnapshotSummary() { Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); Assertions.assertEquals( Long.toString(ts.toInstant().toEpochMilli()), summary.get(VTTS_SNAPSHOT_PROP)); + Assertions.assertEquals(99L, Long.valueOf(summary.get(TX_ID_VALID_THROUGH_PROP))); + Assertions.assertEquals(102L, Long.valueOf(summary.get(MAX_TX_ID__PROP))); } /** @@ -423,7 +425,9 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { OffsetDateTime.ofInstant( Instant.ofEpochMilli(100L), ZoneOffset.UTC))), ImmutableList.of( - new TopicPartitionTransaction(SRC_TOPIC_NAME, 0, 100L))))))); + new TopicPartitionTransaction(SRC_TOPIC_NAME, 0, 100L), + new TopicPartitionTransaction(SRC_TOPIC_NAME, 1, 110L), + new TopicPartitionTransaction(SRC_TOPIC_NAME, 2, 102L))))))); currentControlTopicOffset += 1; } @@ -464,6 +468,13 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { "100", secondSnapshot.summary().get(VTTS_SNAPSHOT_PROP), "Only the most recent snapshot should include vtts in it's summary"); + Assertions.assertEquals( + 99L, + Long.valueOf(secondSnapshot.summary().get(TX_ID_VALID_THROUGH_PROP)), + "The lowest txId processed by all workers -1 should be the txId valid through"); + Assertions.assertEquals(110L, + Long.valueOf(secondSnapshot.summary().get(MAX_TX_ID__PROP)), + "Max txId processed by all workers should be the max txId"); } private void assertCommitTable(int idx, UUID commitId, OffsetDateTime ts) { diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 420ebfad..6d591370 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -69,21 +69,16 @@ public void testStaticRouteTransactionEvent() { when(config.tables()).thenReturn(ImmutableList.of(TABLE_NAME)); when(config.catalogName()).thenReturn("catalog"); Map value = ImmutableMap.of(TRANSACTION_FIELD_NAME, 743); - workerTest(config, value); - } - - @Test - public void testDynamicRouteTransactionEvent() { - IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.dynamicTablesEnabled()).thenReturn(true); - when(config.tablesRouteField()).thenReturn(FIELD_NAME); - when(config.catalogName()).thenReturn("catalog"); + Committable committable = workerTest(config, value); - Map value = ImmutableMap.of(TRANSACTION_FIELD_NAME, TABLE_NAME); - workerTest(config, value); + assertThat( + committable + .txIdsByTopicPartition() + .get(committable.txIdsByTopicPartition().keySet().iterator().next())) + .isEqualTo(743L); } - private void workerTest(IcebergSinkConfig config, Map value) { + private Committable workerTest(IcebergSinkConfig config, Map value) { WriterResult writeResult = new WriterResult( TableIdentifier.parse(TABLE_NAME), @@ -112,5 +107,7 @@ private void workerTest(IcebergSinkConfig config, Map value) { .get(committable.offsetsByTopicPartition().keySet().iterator().next()) .offset()) .isEqualTo(1L); + + return committable; } } From a17f8dd4dbb41db09d5ef9e79cd12107a2e7ddde Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 31 Jul 2024 15:02:07 -0400 Subject: [PATCH 12/29] Clean up and testing --- .../iceberg/connect/events/EventSerializationTest.java | 4 ++++ .../io/tabular/iceberg/connect/channel/CommitterImplTest.java | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java index 9e87dc0a..fd5c4be3 100644 --- a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java +++ b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java @@ -95,6 +95,10 @@ public void testCommitReadySerialization() { assertThat(payload.commitId()).isEqualTo(commitId); assertThat(payload.assignments()).hasSize(2); assertThat(payload.assignments()).allMatch(tp -> tp.topic().equals("topic")); + assertThat(payload.txIds()).hasSize(2); + assertThat(payload.txIds()).allMatch(tp -> tp.topic().equals("topic")); + assertThat(payload.txIds().get(0).txId()).isEqualTo(1L); + assertThat(payload.txIds().get(1).txId()).isNull(); } @Test diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java index c36068d3..3b8a89f4 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java @@ -563,7 +563,7 @@ public void testCommitShouldCommitOffsetsOnlyForPartitionsWeMadeProgressOn() thr () -> new Committable( ImmutableMap.of(sourceTp1, new Offset(100L, 200L)), - ImmutableMap.of(), + ImmutableMap.of(sourceTp1, 999L), ImmutableList.of( new WriterResult(TABLE_1_IDENTIFIER, dataFiles, deleteFiles, partitionStruct))); @@ -601,7 +601,7 @@ public void testCommitShouldCommitOffsetsOnlyForPartitionsWeMadeProgressOn() thr ImmutableMap.of( sourceTp0, Pair.of(null, null), sourceTp1, Pair.of(100L, offsetDateTime(200L))), - ImmutableMap.of()); + ImmutableMap.of(sourceTp1, 999L)); assertThat(producer.consumerGroupOffsetsHistory()).hasSize(2); Map expectedConsumerOffset = From e366c4e67402b6403acebac0cbb239af7a5fda30 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 7 Aug 2024 14:42:30 -0400 Subject: [PATCH 13/29] fix underscores --- .../java/io/tabular/iceberg/connect/channel/Coordinator.java | 4 ++-- .../io/tabular/iceberg/connect/channel/ChannelTestBase.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 95faf2d4..c97f6a53 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -68,8 +68,8 @@ public class Coordinator extends Channel implements AutoCloseable { private static final String COMMIT_ID_SNAPSHOT_PROP = "kafka.connect.commit-id"; private static final String VTTS_SNAPSHOT_PROP = "kafka.connect.vtts"; private static final Duration POLL_DURATION = Duration.ofMillis(1000); - private static final String TXID_VALID_THROUGH_PROP = "txid_valid_through"; - private static final String TXID_MAX_PROP = "txid_max"; + private static final String TXID_VALID_THROUGH_PROP = "txid-valid-through"; + private static final String TXID_MAX_PROP = "txid-max"; private final Catalog catalog; private final IcebergSinkConfig config; diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java index 0869280e..055d7ae6 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java @@ -86,8 +86,8 @@ private InMemoryCatalog initInMemoryCatalog() { protected static final String OFFSETS_SNAPSHOT_PROP = String.format("kafka.connect.offsets.%s.%s", CTL_TOPIC_NAME, CONTROL_CONSUMER_GROUP_ID); protected static final String VTTS_SNAPSHOT_PROP = "kafka.connect.vtts"; - protected static final String TX_ID_VALID_THROUGH_PROP = "txid_valid_through"; - protected static final String MAX_TX_ID__PROP = "txid_max"; + protected static final String TX_ID_VALID_THROUGH_PROP = "txid-valid-through"; + protected static final String MAX_TX_ID__PROP = "txid-max"; @BeforeEach @SuppressWarnings("deprecation") From 3f34fed90f2f220f247eb990203add002a3494ed Mon Sep 17 00:00:00 2001 From: Sungjin Yook Date: Mon, 12 Aug 2024 12:17:50 -0500 Subject: [PATCH 14/29] creating 1st layer partition-by field - source_ts_us --- .../transforms/CustomFieldConstants.java | 25 +++++++++++++++++++ .../connect/transforms/DebeziumTransform.java | 8 ++++++ 2 files changed, 33 insertions(+) create mode 100644 kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CustomFieldConstants.java diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CustomFieldConstants.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CustomFieldConstants.java new file mode 100644 index 00000000..dbffa325 --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/CustomFieldConstants.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.transforms; + +public interface CustomFieldConstants { + String SOURCE_TIMESTAMP_US = "source_ts_us"; + String SOURCE_TIMESTAMP_MS = "source_ts_ms"; + String SOURCE_TIMESTAMP_NS = "source_ts_ns"; +} diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index 39683882..d7481d49 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -110,6 +110,10 @@ private R applyWithSchema(R record) { } newValue.put(CdcConstants.COL_CDC, cdcMetadata); + if (value.getStruct("ts_us") != null) { + newValue.put(CustomFieldConstants.SOURCE_TIMESTAMP_US, new java.util.Date(value.getInt64("ts_us"))); + } + return record.newRecord( record.topic(), record.kafkaPartition(), @@ -155,6 +159,10 @@ private R applySchemaless(R record) { Map newValue = Maps.newHashMap((Map) payload); newValue.put(CdcConstants.COL_CDC, cdcMetadata); + if (value.containsKey("ts_us")) { + newValue.put(CustomFieldConstants.SOURCE_TIMESTAMP_US, value.get("ts_us")); + } + return record.newRecord( record.topic(), record.kafkaPartition(), From e4d1c6f0df73e815acc1b958917a4ac9bf42a1c1 Mon Sep 17 00:00:00 2001 From: Sungjin Yook Date: Mon, 12 Aug 2024 13:48:58 -0500 Subject: [PATCH 15/29] fix gradle format check err --- .../events/TopicPartitionTransaction.java | 1 - .../connect/events/TopicPartitionTxId.java | 20 ++++++++++++++++++- .../events/TransactionDataComplete.java | 18 +++++++++++++++++ 3 files changed, 37 insertions(+), 2 deletions(-) diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java index 48ec177a..ac14d38d 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java @@ -89,7 +89,6 @@ public void put(int i, Object v) { throw new IllegalArgumentException("Unknown field index: " + i); } } - @Override public Object get(int i) { switch (positionToId(i, avroSchema)) { diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java index 22ff33ad..c6c75073 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package io.tabular.iceberg.connect.events; import org.apache.avro.Schema; @@ -92,4 +110,4 @@ public Object get(int i) { throw new UnsupportedOperationException("Unknown field ordinal: " + i); } } -} \ No newline at end of file +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java index b0023bd4..1b0628d5 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package io.tabular.iceberg.connect.events; import org.apache.avro.Schema; From f1b2fb6edbb488a1a0824b40b031f4d8e8f90f4f Mon Sep 17 00:00:00 2001 From: Kristin Greenman Date: Mon, 12 Aug 2024 13:49:08 -0500 Subject: [PATCH 16/29] Fix checkstyle issues --- .../events/TopicPartitionTransaction.java | 1 - .../connect/events/TopicPartitionTxId.java | 20 +++++++++++++++++- .../events/TransactionDataComplete.java | 21 +++++++++++++++++-- 3 files changed, 38 insertions(+), 4 deletions(-) diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java index 48ec177a..ac14d38d 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTransaction.java @@ -89,7 +89,6 @@ public void put(int i, Object v) { throw new IllegalArgumentException("Unknown field index: " + i); } } - @Override public Object get(int i) { switch (positionToId(i, avroSchema)) { diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java index 22ff33ad..c6c75073 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionTxId.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package io.tabular.iceberg.connect.events; import org.apache.avro.Schema; @@ -92,4 +110,4 @@ public Object get(int i) { throw new UnsupportedOperationException("Unknown field ordinal: " + i); } } -} \ No newline at end of file +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java index b0023bd4..571cb059 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TransactionDataComplete.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package io.tabular.iceberg.connect.events; import org.apache.avro.Schema; @@ -12,7 +30,6 @@ import java.util.UUID; import java.util.stream.Collectors; -import static org.apache.iceberg.connect.events.PayloadType.DATA_COMPLETE; public class TransactionDataComplete implements org.apache.iceberg.connect.events.Payload { @@ -72,7 +89,7 @@ public Schema getSchema() { @Override public PayloadType type() { - return DATA_COMPLETE; + return org.apache.iceberg.connect.events.PayloadType.DATA_COMPLETE; } @Override From 37604f04e04cda321cc6d9071618873b232246ac Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 11:42:19 -0400 Subject: [PATCH 17/29] fix nullpointer --- .../main/java/io/tabular/iceberg/connect/data/Utilities.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index c3ee1732..8c4b7d51 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -122,6 +122,9 @@ private static Object loadHadoopConfig(IcebergSinkConfig config) { } public static Object extractFromRecordValue(Object recordValue, String fieldName) { + if (recordValue == null) { + return null; + } String[] fields = fieldName.split("\\."); if (recordValue instanceof Struct) { return getValueFromStruct((Struct) recordValue, fields, 0); @@ -135,6 +138,7 @@ public static Object extractFromRecordValue(Object recordValue, String fieldName public static Long extractTxIdFromRecordValue(Object recordValue, String fieldName) { Object txId = extractFromRecordValue(recordValue, fieldName); + if (txId instanceof Number) { return ((Number) txId).longValue(); } else { From 1da9fc2fa0b35e088dd68eb13cf37a31779427ff Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 12:36:16 -0400 Subject: [PATCH 18/29] Add Debugs statements --- .../iceberg/connect/channel/Coordinator.java | 21 +++++++++++++------ .../iceberg/connect/channel/Worker.java | 1 + .../iceberg/connect/data/IcebergWriter.java | 2 ++ .../iceberg/connect/data/Utilities.java | 12 ++++++++--- 4 files changed, 27 insertions(+), 9 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index c97f6a53..138f97e1 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -42,6 +42,7 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Catalog; @@ -232,8 +233,8 @@ private void commitToTable( if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); } else { - String txIdValidThrough = Long.toString(Utilities.calculateTxIdValidThrough(highestTxIdPerPartition())); - String maxTxId = Long.toString(Utilities.getMaxTxId(highestTxIdPerPartition())); + long txIdValidThrough = Utilities.calculateTxIdValidThrough(highestTxIdPerPartition()); + long maxTxId = Utilities.getMaxTxId(highestTxIdPerPartition()); if (deleteFiles.isEmpty()) { Transaction transaction = table.newTransaction(); @@ -254,8 +255,7 @@ private void commitToTable( if (vtts != null) { appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - appendOp.set(TXID_VALID_THROUGH_PROP, txIdValidThrough); - appendOp.set(TXID_MAX_PROP, maxTxId); + addTxIdDataToSnapshot(appendOp, txIdValidThrough, maxTxId); } appendOp.commit(); @@ -270,8 +270,7 @@ private void commitToTable( if (vtts != null) { deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - deltaOp.set(TXID_VALID_THROUGH_PROP, txIdValidThrough); - deltaOp.set(TXID_MAX_PROP, maxTxId); + addTxIdDataToSnapshot(deltaOp, txIdValidThrough, maxTxId); dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); deltaOp.commit(); @@ -297,6 +296,16 @@ private void commitToTable( } } + private void addTxIdDataToSnapshot(SnapshotUpdate operation, long txIdValidThrough, long maxTxId) { + if (txIdValidThrough > 0 && maxTxId > 0) { + operation.set(TXID_VALID_THROUGH_PROP, Long.toString(txIdValidThrough)); + operation.set(TXID_MAX_PROP, Long.toString(maxTxId)); + LOG.debug("Added transaction data to snapshot: validThrough={}, max={}", txIdValidThrough, maxTxId); + } else { + LOG.warn("No transaction data to add to snapshot"); + } + } + private Snapshot latestSnapshot(Table table, String branch) { if (branch == null) { return table.currentSnapshot(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index ae2ffde5..852db979 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -101,6 +101,7 @@ private void save(SinkRecord record) { Long txId = Utilities.extractTxIdFromRecordValue(record.value(), COL_TXID); if (txId != null) { + LOG.debug("Found transaction id {} in record", txId); sourceTxIds.put( new TopicPartition(record.topic(), record.kafkaPartition()), txId); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java index f0952bf3..b9fada00 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java @@ -70,6 +70,7 @@ public void write(SinkRecord record) { if (cdcField == null) { writer.write(row); } else { + LOG.debug("Extracting CDC field: {} from Record {}", cdcField, record.value()); Operation op = extractCdcOperation(record.value(), cdcField); writer.write(new RecordWrapper(row, op)); } @@ -109,6 +110,7 @@ private Record convertToRow(SinkRecord record) { private Operation extractCdcOperation(Object recordValue, String cdcField) { Object opValue = Utilities.extractFromRecordValue(recordValue, cdcField); + LOG.debug("Processing cdc field value: {}", opValue); if (opValue == null) { return Operation.INSERT; } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index 8c4b7d51..6c505d35 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -139,9 +139,15 @@ public static Object extractFromRecordValue(Object recordValue, String fieldName public static Long extractTxIdFromRecordValue(Object recordValue, String fieldName) { Object txId = extractFromRecordValue(recordValue, fieldName); - if (txId instanceof Number) { - return ((Number) txId).longValue(); - } else { + if (txId == null) { + LOG.debug("Transaction ID field not found in recordValue {}", recordValue); + return null; + } + + try { + return Long.parseLong(txId.toString().trim()); + } catch (NumberFormatException e) { + LOG.warn("Invalid transaction ID value: {}", txId); return null; } } From 8411e50b16594292982c0219f90497045c677a96 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 13:07:26 -0400 Subject: [PATCH 19/29] Add Debugs statements --- .../main/java/io/tabular/iceberg/connect/data/Utilities.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index 6c505d35..b87891ec 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -301,9 +301,11 @@ public static void close(C closeable) { public static Long calculateTxIdValidThrough(Map highestTxIdPerPartition) { if (highestTxIdPerPartition.isEmpty()) { + LOG.debug("Transaction Map is empty, returning 0"); return 0L; } + LOG.debug("Transaction Map contains {} entries", highestTxIdPerPartition.size()); // Find the minimum value in the map, as it represents the highest transaction ID // that is common across all partitions long minValue = Collections.min(highestTxIdPerPartition.values()); From 355a5a813c377fe3c29b3ac3c9cb0f0ff6fee2f9 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 13:19:25 -0400 Subject: [PATCH 20/29] Add Debugs statements --- .../io/tabular/iceberg/connect/channel/Coordinator.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 138f97e1..3a8f8510 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -129,11 +129,10 @@ private boolean receive(Envelope envelope) { if (envelope.event().payload() instanceof TransactionDataComplete) { TransactionDataComplete payload = (TransactionDataComplete) envelope.event().payload(); List txIds = payload.txIds(); - if (txIds != null) { + LOG.debug("Received transaction data complete event with {} txIds", txIds.size()); txIds.forEach( txId -> highestTxIdPerPartition().put(txId.partition(), Math.max(highestTxIdPerPartition().getOrDefault(txId.partition(), 0L), txId.txId()))); - } } if (commitState.isCommitReady(totalPartitionCount)) { commit(false); @@ -255,7 +254,7 @@ private void commitToTable( if (vtts != null) { appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - addTxIdDataToSnapshot(appendOp, txIdValidThrough, maxTxId); + addTxDataToSnapshot(appendOp, txIdValidThrough, maxTxId); } appendOp.commit(); @@ -270,7 +269,7 @@ private void commitToTable( if (vtts != null) { deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } - addTxIdDataToSnapshot(deltaOp, txIdValidThrough, maxTxId); + addTxDataToSnapshot(deltaOp, txIdValidThrough, maxTxId); dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); deltaOp.commit(); @@ -296,7 +295,7 @@ private void commitToTable( } } - private void addTxIdDataToSnapshot(SnapshotUpdate operation, long txIdValidThrough, long maxTxId) { + private void addTxDataToSnapshot(SnapshotUpdate operation, long txIdValidThrough, long maxTxId) { if (txIdValidThrough > 0 && maxTxId > 0) { operation.set(TXID_VALID_THROUGH_PROP, Long.toString(txIdValidThrough)); operation.set(TXID_MAX_PROP, Long.toString(maxTxId)); From c35cfbcf9e90ca4319a48f020c67a0f6ec631921 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 15:18:08 -0400 Subject: [PATCH 21/29] change txid parsing --- .../iceberg/connect/data/Utilities.java | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index b87891ec..5fca2e96 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -132,7 +132,7 @@ public static Object extractFromRecordValue(Object recordValue, String fieldName return getValueFromMap((Map) recordValue, fields, 0); } else { throw new UnsupportedOperationException( - "Cannot extract value from type: " + recordValue.getClass().getName()); + "Cannot extract value from type: " + recordValue.getClass().getName()); } } @@ -147,7 +147,7 @@ public static Long extractTxIdFromRecordValue(Object recordValue, String fieldNa try { return Long.parseLong(txId.toString().trim()); } catch (NumberFormatException e) { - LOG.warn("Invalid transaction ID value: {}", txId); + LOG.warn("Failed to parse transaction ID: {}", txId, e); return null; } } @@ -158,9 +158,13 @@ private static Object getValueFromStruct(Struct struct, String[] fields, int idx if (value == null || idx == fields.length - 1) { return value; } - - Preconditions.checkState(value instanceof Struct, "Expected a struct type"); - return getValueFromStruct((Struct) value, fields, idx + 1); + if (value instanceof Struct) { + return getValueFromStruct((Struct) value, fields, idx + 1); + } else if (value instanceof Map) { + return getValueFromMap((Map) value, fields, idx + 1); + } else { + return value; + } } private static Object getValueFromMap(Map map, String[] fields, int idx) { @@ -169,9 +173,13 @@ private static Object getValueFromMap(Map map, String[] fields, int idx) { if (value == null || idx == fields.length - 1) { return value; } - - Preconditions.checkState(value instanceof Map, "Expected a map type"); - return getValueFromMap((Map) value, fields, idx + 1); + if (value instanceof Struct) { + return getValueFromStruct((Struct) value, fields, idx + 1); + } else if (value instanceof Map) { + return getValueFromMap((Map) value, fields, idx + 1); + } else { + return value; + } } public static TaskWriter createTableWriter( From f3531c9c1f28b14eebcb0a6732b6093f44bf0e93 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 15:30:15 -0400 Subject: [PATCH 22/29] change txid parsing as json --- .../iceberg/connect/data/Utilities.java | 27 ++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index 5fca2e96..a4d8711e 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -24,6 +24,8 @@ import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES; import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import io.tabular.iceberg.connect.IcebergSinkConfig; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -65,6 +67,7 @@ public class Utilities { private static final Logger LOG = LoggerFactory.getLogger(Utilities.class.getName()); private static final List HADOOP_CONF_FILES = ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); + private static final ObjectMapper objectMapper = new ObjectMapper(); public static Catalog loadCatalog(IcebergSinkConfig config) { return CatalogUtil.buildIcebergCatalog( @@ -141,7 +144,7 @@ public static Long extractTxIdFromRecordValue(Object recordValue, String fieldNa if (txId == null) { LOG.debug("Transaction ID field not found in recordValue {}", recordValue); - return null; + return extractTxIdFromRecordValueAsJson(recordValue); } try { @@ -152,6 +155,28 @@ public static Long extractTxIdFromRecordValue(Object recordValue, String fieldNa } } + public static Long extractTxIdFromRecordValueAsJson(Object recordValue) { + if (recordValue == null) { + LOG.debug("Record value is null"); + return null; + } + + try { + JsonNode rootNode = objectMapper.readTree(recordValue.toString()); + JsonNode txIdNode = rootNode.at("/_cdc/txid"); + + if (txIdNode.isMissingNode()) { + LOG.debug("Transaction ID field not found in recordValue {}", recordValue); + return null; + } + + return txIdNode.asLong(); + } catch (IOException e) { + LOG.warn("Failed to parse record value as JSON: {}", recordValue, e); + return null; + } + } + private static Object getValueFromStruct(Struct struct, String[] fields, int idx) { Preconditions.checkArgument(idx < fields.length, "Invalid field index"); Object value = struct.get(fields[idx]); From 20d61188d2bcaf8aaab23e40643001b82650fd13 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 15:42:42 -0400 Subject: [PATCH 23/29] change txid parsing as json --- .../java/io/tabular/iceberg/connect/data/Utilities.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index a4d8711e..c4b381e5 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -162,11 +162,16 @@ public static Long extractTxIdFromRecordValueAsJson(Object recordValue) { } try { - JsonNode rootNode = objectMapper.readTree(recordValue.toString()); + String jsonString = recordValue.toString() + .replaceAll("([a-zA-Z0-9_]+)=", "\"$1\":") + .replaceAll("=", ":") + .replaceAll("'", "\""); + + JsonNode rootNode = objectMapper.readTree(jsonString); JsonNode txIdNode = rootNode.at("/_cdc/txid"); if (txIdNode.isMissingNode()) { - LOG.debug("Transaction ID field not found in recordValue {}", recordValue); + LOG.debug("Transaction ID node not found in recordValue json object {}", recordValue); return null; } From 45e95e7d5984030e14063c948868dea4ccb8a70d Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 15:54:37 -0400 Subject: [PATCH 24/29] change txid parsing as json --- .../main/java/io/tabular/iceberg/connect/data/Utilities.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index c4b381e5..6686d068 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -144,7 +144,7 @@ public static Long extractTxIdFromRecordValue(Object recordValue, String fieldNa if (txId == null) { LOG.debug("Transaction ID field not found in recordValue {}", recordValue); - return extractTxIdFromRecordValueAsJson(recordValue); + return extractTxIdFromRecordValueAsJson(recordValue); } try { @@ -164,6 +164,7 @@ public static Long extractTxIdFromRecordValueAsJson(Object recordValue) { try { String jsonString = recordValue.toString() .replaceAll("([a-zA-Z0-9_]+)=", "\"$1\":") + .replaceAll("=([a-zA-Z0-9_]+)", ":\"$1\"") .replaceAll("=", ":") .replaceAll("'", "\""); From 7a1ed7efe59f99f563a1beeeaacfe69e4dda775d Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 16:33:35 -0400 Subject: [PATCH 25/29] Parse txid with regex --- .../iceberg/connect/channel/Worker.java | 2 +- .../iceberg/connect/data/Utilities.java | 52 ++++++++----------- .../iceberg/connect/data/UtilitiesTest.java | 8 +++ 3 files changed, 32 insertions(+), 30 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 852db979..267745d2 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -43,7 +43,7 @@ class Worker implements Writer, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Worker.class); - private static final String COL_TXID = "_cdc_txid"; + private static final String COL_TXID = "txid"; private final IcebergSinkConfig config; private final IcebergWriterFactory writerFactory; private final Map writers; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index 6686d068..f6ebd019 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -24,8 +24,6 @@ import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES; import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; import io.tabular.iceberg.connect.IcebergSinkConfig; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -67,7 +65,6 @@ public class Utilities { private static final Logger LOG = LoggerFactory.getLogger(Utilities.class.getName()); private static final List HADOOP_CONF_FILES = ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); - private static final ObjectMapper objectMapper = new ObjectMapper(); public static Catalog loadCatalog(IcebergSinkConfig config) { return CatalogUtil.buildIcebergCatalog( @@ -139,48 +136,45 @@ public static Object extractFromRecordValue(Object recordValue, String fieldName } } - public static Long extractTxIdFromRecordValue(Object recordValue, String fieldName) { + public static Long extractTxIdFromRecordValueOG(Object recordValue, String fieldName) { Object txId = extractFromRecordValue(recordValue, fieldName); if (txId == null) { - LOG.debug("Transaction ID field not found in recordValue {}", recordValue); - return extractTxIdFromRecordValueAsJson(recordValue); + LOG.debug("Transaction ID field not found in record {}", recordValue); + return null; } try { return Long.parseLong(txId.toString().trim()); } catch (NumberFormatException e) { - LOG.warn("Failed to parse transaction ID: {}", txId, e); + LOG.warn("Invalid transaction ID value: {}", txId); return null; } } - public static Long extractTxIdFromRecordValueAsJson(Object recordValue) { - if (recordValue == null) { - LOG.debug("Record value is null"); - return null; - } + public static Long extractTxIdFromRecordValue(Object recordValue, String fieldName) { - try { - String jsonString = recordValue.toString() - .replaceAll("([a-zA-Z0-9_]+)=", "\"$1\":") - .replaceAll("=([a-zA-Z0-9_]+)", ":\"$1\"") - .replaceAll("=", ":") - .replaceAll("'", "\""); - - JsonNode rootNode = objectMapper.readTree(jsonString); - JsonNode txIdNode = rootNode.at("/_cdc/txid"); - - if (txIdNode.isMissingNode()) { - LOG.debug("Transaction ID node not found in recordValue json object {}", recordValue); - return null; + String recordStr = recordValue.toString(); + String fieldValue = extractFieldValue(recordStr, fieldName); + if (fieldValue != null) { + try { + return Long.parseLong(fieldValue); + } catch (NumberFormatException e) { + LOG.error("Failed to parse txid value: {}", fieldValue, e); + } } + return null; + } - return txIdNode.asLong(); - } catch (IOException e) { - LOG.warn("Failed to parse record value as JSON: {}", recordValue, e); - return null; + private static String extractFieldValue(String recordStr, String fieldName) { + String regex = fieldName.replace(".", "\\.") + "=([^,}]+)"; + java.util.regex.Pattern pattern = java.util.regex.Pattern.compile(regex); + java.util.regex.Matcher matcher = pattern.matcher(recordStr); + + if (matcher.find()) { + return matcher.group(1); } + return null; } private static Object getValueFromStruct(Struct struct, String[] fields, int idx) { diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java index 42884708..f3324085 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java @@ -157,4 +157,12 @@ public void testExtractFromRecordValueMapNested() { Object result = Utilities.extractFromRecordValue(val, "data.id.key"); assertThat(result).isEqualTo(123L); } + + @Test + public void testExtractTxIdFromRecordValue() { + String recordValue = "{product=theora-tools, rowcreatedtimestamp=1723595528604435, source_ts_us=Wed Aug 14 19:48:08 UTC 2024, certainty=1.0, description=Linux theora-tools 1:1.1.1-21.el8, _pkhash=d863eaee4c697faa9a33e2c5cd164e52df6b68b20047cc83359f193181794c79, type=, version=1:1.1.1-21.el8, orgid=00c67732-fbb0-420b-8ebd-7fedbdb99466, rowupdatedtimestamp=1723595528604435, _cdc={op=I, offset=11474746, txid=1555378374, source=vm_ingress.asset_software, ts=1723664888037, target=vm_ingress.asset_software}, assetid=407fec2f-001c-4596-a3ac-683328c47aee-default-asset-1364238, vendor=Linux, family=}"; + Long expectedTxId = 1555378374L; + Long actualTxId = Utilities.extractTxIdFromRecordValue(recordValue, "txid"); + assertThat(actualTxId).isEqualTo(expectedTxId); + } } From ef75261fa590946b7cbac7ca3c6d4f8875bbcce8 Mon Sep 17 00:00:00 2001 From: dtobon Date: Wed, 14 Aug 2024 17:03:16 -0400 Subject: [PATCH 26/29] Parse txid with regex --- .../iceberg/connect/data/Utilities.java | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index f6ebd019..06adb14a 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -136,24 +136,10 @@ public static Object extractFromRecordValue(Object recordValue, String fieldName } } - public static Long extractTxIdFromRecordValueOG(Object recordValue, String fieldName) { - Object txId = extractFromRecordValue(recordValue, fieldName); - - if (txId == null) { - LOG.debug("Transaction ID field not found in record {}", recordValue); - return null; - } - - try { - return Long.parseLong(txId.toString().trim()); - } catch (NumberFormatException e) { - LOG.warn("Invalid transaction ID value: {}", txId); + public static Long extractTxIdFromRecordValue(Object recordValue, String fieldName) { + if (recordValue == null) { return null; } - } - - public static Long extractTxIdFromRecordValue(Object recordValue, String fieldName) { - String recordStr = recordValue.toString(); String fieldValue = extractFieldValue(recordStr, fieldName); if (fieldValue != null) { From 865b3083f98263abd730a55cb2de87b78dcbffe2 Mon Sep 17 00:00:00 2001 From: dtobon Date: Thu, 15 Aug 2024 13:10:01 -0400 Subject: [PATCH 27/29] Cleanup and logging updates --- .../iceberg/connect/channel/Coordinator.java | 2 +- .../iceberg/connect/data/Utilities.java | 24 +++++++------------ .../iceberg/connect/data/UtilitiesTest.java | 2 +- 3 files changed, 10 insertions(+), 18 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 3a8f8510..ca2e767d 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -299,7 +299,7 @@ private void addTxDataToSnapshot(SnapshotUpdate operation, long txIdValidThro if (txIdValidThrough > 0 && maxTxId > 0) { operation.set(TXID_VALID_THROUGH_PROP, Long.toString(txIdValidThrough)); operation.set(TXID_MAX_PROP, Long.toString(maxTxId)); - LOG.debug("Added transaction data to snapshot: validThrough={}, max={}", txIdValidThrough, maxTxId); + LOG.info("Added transaction data to snapshot: validThrough={}, max={}", txIdValidThrough, maxTxId); } else { LOG.warn("No transaction data to add to snapshot"); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java index 06adb14a..2d05697c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java @@ -132,7 +132,7 @@ public static Object extractFromRecordValue(Object recordValue, String fieldName return getValueFromMap((Map) recordValue, fields, 0); } else { throw new UnsupportedOperationException( - "Cannot extract value from type: " + recordValue.getClass().getName()); + "Cannot extract value from type: " + recordValue.getClass().getName()); } } @@ -146,7 +146,7 @@ public static Long extractTxIdFromRecordValue(Object recordValue, String fieldNa try { return Long.parseLong(fieldValue); } catch (NumberFormatException e) { - LOG.error("Failed to parse txid value: {}", fieldValue, e); + LOG.error("Failed to parse fieldName value: {}", fieldValue, e); } } return null; @@ -169,13 +169,9 @@ private static Object getValueFromStruct(Struct struct, String[] fields, int idx if (value == null || idx == fields.length - 1) { return value; } - if (value instanceof Struct) { - return getValueFromStruct((Struct) value, fields, idx + 1); - } else if (value instanceof Map) { - return getValueFromMap((Map) value, fields, idx + 1); - } else { - return value; - } + + Preconditions.checkState(value instanceof Struct, "Expected a struct type"); + return getValueFromStruct((Struct) value, fields, idx + 1); } private static Object getValueFromMap(Map map, String[] fields, int idx) { @@ -184,13 +180,9 @@ private static Object getValueFromMap(Map map, String[] fields, int idx) { if (value == null || idx == fields.length - 1) { return value; } - if (value instanceof Struct) { - return getValueFromStruct((Struct) value, fields, idx + 1); - } else if (value instanceof Map) { - return getValueFromMap((Map) value, fields, idx + 1); - } else { - return value; - } + + Preconditions.checkState(value instanceof Map, "Expected a map type"); + return getValueFromMap((Map) value, fields, idx + 1); } public static TaskWriter createTableWriter( diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java index f3324085..b3fd75dc 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/UtilitiesTest.java @@ -160,7 +160,7 @@ public void testExtractFromRecordValueMapNested() { @Test public void testExtractTxIdFromRecordValue() { - String recordValue = "{product=theora-tools, rowcreatedtimestamp=1723595528604435, source_ts_us=Wed Aug 14 19:48:08 UTC 2024, certainty=1.0, description=Linux theora-tools 1:1.1.1-21.el8, _pkhash=d863eaee4c697faa9a33e2c5cd164e52df6b68b20047cc83359f193181794c79, type=, version=1:1.1.1-21.el8, orgid=00c67732-fbb0-420b-8ebd-7fedbdb99466, rowupdatedtimestamp=1723595528604435, _cdc={op=I, offset=11474746, txid=1555378374, source=vm_ingress.asset_software, ts=1723664888037, target=vm_ingress.asset_software}, assetid=407fec2f-001c-4596-a3ac-683328c47aee-default-asset-1364238, vendor=Linux, family=}"; + String recordValue = "{product=test, rowcreatedtimestamp=1723595528604435, source_ts_us=Wed Aug 14 19:48:08 UTC 2024, certainty=1.0, description=Linux theora-tools 1:1.1.1-21.el8, _pkhash=d863eaee4c697faa9a33e2c5cd164e52df6b68b20047cc83359f193181794c79, type=, version=1:1.1.1-21.el8, orgid=00c67732-fbb0-420b-8ebd-7fedbdb12345, rowupdatedtimestamp=1723595528604435, _cdc={op=I, offset=11474746, txid=1555378374, source=test.test_table, ts=1723664888037, target=vm_ingress.asset_software}, assetid=407fec2f-001c-4596-a3ac-683328c47aee-default-asset-1364238, vendor=Linux, family=}"; Long expectedTxId = 1555378374L; Long actualTxId = Utilities.extractTxIdFromRecordValue(recordValue, "txid"); assertThat(actualTxId).isEqualTo(expectedTxId); From 4082fd442792395da81e448a98da418f899693a0 Mon Sep 17 00:00:00 2001 From: dtobon Date: Tue, 22 Oct 2024 14:44:45 -0400 Subject: [PATCH 28/29] Handle TX ID rollover --- .../iceberg/connect/channel/Coordinator.java | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index ca2e767d..ee9b3c68 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -130,9 +130,9 @@ private boolean receive(Envelope envelope) { TransactionDataComplete payload = (TransactionDataComplete) envelope.event().payload(); List txIds = payload.txIds(); LOG.debug("Received transaction data complete event with {} txIds", txIds.size()); - txIds.forEach( - txId -> highestTxIdPerPartition().put(txId.partition(), - Math.max(highestTxIdPerPartition().getOrDefault(txId.partition(), 0L), txId.txId()))); + txIds.forEach( + txId -> highestTxIdPerPartition().put(txId.partition(), + compareTxIds(highestTxIdPerPartition().getOrDefault(txId.partition(), 0L), txId.txId()))); } if (commitState.isCommitReady(totalPartitionCount)) { commit(false); @@ -142,6 +142,25 @@ private boolean receive(Envelope envelope) { return false; } + /** + * The rollover handling is managed by the compareTxIds method. + * This method compares the current transaction ID (currentTxId) with the new transaction ID (newTxId) and accounts for the rollover scenario. + * + * Rollover Detection: The method checks if the newTxId is less than the currentTxId and if the difference between them is greater than half of Integer.MAX_VALUE. + * This condition indicates that the newTxId has rolled over and is actually higher than the currentTxId. + * Return Value: If the rollover condition is met, the method returns the newTxId as the higher value. + * Otherwise, it returns the maximum of currentTxId and newTxId. + * @param currentTxId + * @param newTxId + * @return + */ + private long compareTxIds(long currentTxId, long newTxId) { + if (newTxId < currentTxId && currentTxId - newTxId > Integer.MAX_VALUE / 2) { + return newTxId; // newTxId has rolled over and is actually higher + } + return Math.max(currentTxId, newTxId); + } + private void commit(boolean partialCommit) { try { LOG.info("Processing commit after responses for {}, isPartialCommit {}",commitState.currentCommitId(), partialCommit); From 34171410c55df2ce81c6243128b18ffe1b793d81 Mon Sep 17 00:00:00 2001 From: dtobon Date: Tue, 22 Oct 2024 18:40:50 -0400 Subject: [PATCH 29/29] Handle TX ID rollover --- .../iceberg/connect/channel/Coordinator.java | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index ee9b3c68..f50184ef 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -145,22 +145,32 @@ private boolean receive(Envelope envelope) { /** * The rollover handling is managed by the compareTxIds method. * This method compares the current transaction ID (currentTxId) with the new transaction ID (newTxId) and accounts for the rollover scenario. - * + *

* Rollover Detection: The method checks if the newTxId is less than the currentTxId and if the difference between them is greater than half of Integer.MAX_VALUE. * This condition indicates that the newTxId has rolled over and is actually higher than the currentTxId. * Return Value: If the rollover condition is met, the method returns the newTxId as the higher value. * Otherwise, it returns the maximum of currentTxId and newTxId. - * @param currentTxId - * @param newTxId - * @return + *

+ * PostgreSQL uses a 32-bit unsigned integer for transaction IDs, which means the wraparound occurs at 2^32 (4,294,967,296). + * We are using 2^31 (2,147,483,648) to detect the wraparound correctly. + * + * @param currentTxId current transaction ID + * @param newTxId new transaction ID + * @return the higher of the two transaction IDs accounting for the rollover scenario */ private long compareTxIds(long currentTxId, long newTxId) { - if (newTxId < currentTxId && currentTxId - newTxId > Integer.MAX_VALUE / 2) { - return newTxId; // newTxId has rolled over and is actually higher + long wraparoundThreshold = 4294967296L; // 2^32 (PostgreSQL wraparound point) + + if ((newTxId > currentTxId && newTxId - currentTxId <= wraparoundThreshold / 2) || + (newTxId < currentTxId && currentTxId - newTxId > wraparoundThreshold / 2)) { + // Wraparound detected: newTxId is actually higher after wrapping around + return newTxId; } + return Math.max(currentTxId, newTxId); } + private void commit(boolean partialCommit) { try { LOG.info("Processing commit after responses for {}, isPartialCommit {}",commitState.currentCommitId(), partialCommit);