fqaiser94 commented on code in PR #10351:
URL: https://github.com/apache/iceberg/pull/10351#discussion_r1663394084


##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.data.Offset;
+import org.apache.iceberg.connect.data.SinkWriter;
+import org.apache.iceberg.connect.data.SinkWriterResult;
+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;
+import org.apache.iceberg.connect.events.StartCommit;
+import org.apache.iceberg.connect.events.TableReference;
+import org.apache.iceberg.connect.events.TopicPartitionOffset;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+
+class Worker extends Channel {
+
+  private final IcebergSinkConfig config;
+  private final SinkTaskContext context;
+  private final SinkWriter sinkWriter;
+
+  Worker(
+      IcebergSinkConfig config,
+      KafkaClientFactory clientFactory,
+      SinkWriter sinkWriter,
+      SinkTaskContext context) {
+    // pass transient consumer group ID to which we never commit offsets
+    super(
+        "worker",
+        IcebergSinkConfig.DEFAULT_CONTROL_GROUP_PREFIX + UUID.randomUUID(),

Review Comment:
   Can we put the connect consumer group id in here somewhere? 
   Otherwise, it's impossible for users  to map a random UUIDs back to a 
specific connector if necessary. 



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+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.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.events.CommitComplete;
+import org.apache.iceberg.connect.events.CommitToTable;
+import org.apache.iceberg.connect.events.DataWritten;
+import org.apache.iceberg.connect.events.Event;
+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.Maps;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.kafka.clients.admin.MemberDescription;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class Coordinator extends Channel {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Coordinator.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  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.ofSeconds(1);
+
+  private final Catalog catalog;
+  private final IcebergSinkConfig config;
+  private final int totalPartitionCount;
+  private final String snapshotOffsetsProp;
+  private final ExecutorService exec;
+  private final CommitState commitState;
+
+  Coordinator(
+      Catalog catalog,
+      IcebergSinkConfig config,
+      Collection<MemberDescription> members,
+      KafkaClientFactory clientFactory,
+      SinkTaskContext context) {
+    // pass consumer group ID to which we commit low watermark offsets
+    super("coordinator", config.connectGroupId() + "-coord", config, 
clientFactory, context);
+
+    this.catalog = catalog;
+    this.config = config;
+    this.totalPartitionCount =
+        members.stream().mapToInt(desc -> 
desc.assignment().topicPartitions().size()).sum();
+    this.snapshotOffsetsProp =
+        String.format(
+            "kafka.connect.offsets.%s.%s", config.controlTopic(), 
config.connectGroupId());
+    this.exec = ThreadPools.newWorkerPool("iceberg-committer", 
config.commitThreads());

Review Comment:
   we never shut this thread pool down. Is that an issue?



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+class KafkaClientFactory {
+  private final Map<String, String> kafkaProps;
+
+  KafkaClientFactory(Map<String, String> kafkaProps) {
+    this.kafkaProps = kafkaProps;
+  }
+
+  Producer<String, byte[]> createProducer(String transactionalId) {
+    Map<String, Object> producerProps = Maps.newHashMap(kafkaProps);
+    producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
+    producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, 
UUID.randomUUID().toString());
+    KafkaProducer<String, byte[]> result =
+        new KafkaProducer<>(producerProps, new StringSerializer(), new 
ByteArraySerializer());
+    result.initTransactions();
+    return result;
+  }
+
+  Consumer<String, byte[]> createConsumer(String consumerGroupId) {
+    Map<String, Object> consumerProps = Maps.newHashMap(kafkaProps);
+    consumerProps.putIfAbsent(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, 
"latest");
+    consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+    consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed");
+    consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupId);
+    consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, 
UUID.randomUUID().toString());

Review Comment:
   ```suggestion
   ```
   
   I would avoid randomly generating client ID config since they're used for 
rate limiting in Kafka. 
   I would just remove this line entirely (if users wish to set a ClientID they 
can do so via the connector kafka configs). 



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.data.Offset;
+import org.apache.iceberg.connect.events.AvroUtil;
+import org.apache.iceberg.connect.events.Event;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class Channel {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Channel.class);
+
+  private final String controlTopic;
+  private final String connectGroupId;
+  private final Producer<String, byte[]> producer;
+  private final Consumer<String, byte[]> consumer;
+  private final SinkTaskContext context;
+  private final Admin admin;
+  private final Map<Integer, Long> controlTopicOffsets = Maps.newHashMap();
+  private final String producerId;
+
+  Channel(
+      String name,
+      String consumerGroupId,
+      IcebergSinkConfig config,
+      KafkaClientFactory clientFactory,
+      SinkTaskContext context) {
+    this.controlTopic = config.controlTopic();
+    this.connectGroupId = config.connectGroupId();
+    this.context = context;
+
+    String transactionalId = name + config.transactionalSuffix();
+    this.producer = clientFactory.createProducer(transactionalId);
+    this.consumer = clientFactory.createConsumer(consumerGroupId);
+    this.admin = clientFactory.createAdmin();
+
+    this.producerId = UUID.randomUUID().toString();
+  }
+
+  protected void send(Event event) {
+    send(ImmutableList.of(event), ImmutableMap.of());
+  }
+
+  @SuppressWarnings("FutureReturnValueIgnored")
+  protected void send(List<Event> events, Map<TopicPartition, Offset> 
sourceOffsets) {
+    Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = Maps.newHashMap();
+    sourceOffsets.forEach((k, v) -> offsetsToCommit.put(k, new 
OffsetAndMetadata(v.offset())));
+
+    List<ProducerRecord<String, byte[]>> recordList =
+        events.stream()
+            .map(
+                event -> {
+                  LOG.info("Sending event of type: {}", event.type().name());
+                  byte[] data = AvroUtil.encode(event);
+                  // key by producer ID to keep event order
+                  return new ProducerRecord<>(controlTopic, producerId, data);
+                })
+            .collect(Collectors.toList());
+
+    synchronized (producer) {
+      producer.beginTransaction();
+      try {
+        // NOTE: we shouldn't call get() on the future in a transactional 
context,
+        // see docs for org.apache.kafka.clients.producer.KafkaProducer
+        recordList.forEach(producer::send);
+        if (!sourceOffsets.isEmpty()) {
+          producer.sendOffsetsToTransaction(
+              offsetsToCommit, KafkaUtils.consumerGroupMetadata(context, 
connectGroupId));

Review Comment:
   ```suggestion
                 offsetsToCommit, this.consumerGroupMetadata);
   ```
   
   We only need to extract consumer-group-metadata once, we should just do that 
in the constructor and make it a class field. 
   
   The consumer-group-metadata contains the consumer-generation which is used 
by Kafka brokers for zombie fencing. I'm probably over-thinking this massively 
but I'm worried that the internal KafkaConsumer might rebalance, get the new 
generation, and then a zombie task could end up successfully committing a 
transaction resulting in duplicates. 
   
   Regardless, I would move this to a class field as that is definitely safe. 



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.time.OffsetDateTime;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.events.DataComplete;
+import org.apache.iceberg.connect.events.DataWritten;
+import org.apache.iceberg.connect.events.TableReference;
+import org.apache.iceberg.connect.events.TopicPartitionOffset;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class CommitState {
+  private static final Logger LOG = LoggerFactory.getLogger(CommitState.class);
+
+  private final List<Envelope> commitBuffer = Lists.newArrayList();
+  private final List<DataComplete> readyBuffer = Lists.newArrayList();
+  private long startTime;
+  private UUID currentCommitId;
+  private final IcebergSinkConfig config;
+
+  CommitState(IcebergSinkConfig config) {
+    this.config = config;
+  }
+
+  void addResponse(Envelope envelope) {
+    commitBuffer.add(envelope);
+    if (!isCommitInProgress()) {
+      DataWritten dataWritten = (DataWritten) envelope.event().payload();
+      LOG.warn(
+          "Received commit response when no commit in progress, this can 
happen during recovery. Commit ID: {}",
+          dataWritten.commitId());
+    }
+  }
+
+  void addReady(Envelope envelope) {
+    DataComplete dataComplete = (DataComplete) envelope.event().payload();
+    readyBuffer.add(dataComplete);
+    if (!isCommitInProgress()) {
+      LOG.warn(
+          "Received commit ready when no commit in progress, this can happen 
during recovery. Commit ID: {}",
+          dataComplete.commitId());
+    }
+  }
+
+  UUID currentCommitId() {
+    return currentCommitId;
+  }
+
+  boolean isCommitInProgress() {
+    return currentCommitId != null;
+  }
+
+  boolean isCommitIntervalReached() {
+    if (startTime == 0) {
+      startTime = System.currentTimeMillis();
+    }
+
+    return (!isCommitInProgress()
+        && System.currentTimeMillis() - startTime >= 
config.commitIntervalMs());
+  }
+
+  void startNewCommit() {
+    currentCommitId = UUID.randomUUID();
+    startTime = System.currentTimeMillis();
+  }
+
+  void endCurrentCommit() {
+    readyBuffer.clear();
+    currentCommitId = null;
+  }
+
+  void clearResponses() {
+    commitBuffer.clear();
+  }
+
+  boolean isCommitTimedOut() {
+    if (!isCommitInProgress()) {
+      return false;
+    }
+
+    if (System.currentTimeMillis() - startTime > config.commitTimeoutMs()) {
+      LOG.info("Commit timeout reached");

Review Comment:
   Please interpolate the commit id in here as well. 



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+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.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.events.CommitComplete;
+import org.apache.iceberg.connect.events.CommitToTable;
+import org.apache.iceberg.connect.events.DataWritten;
+import org.apache.iceberg.connect.events.Event;
+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.Maps;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.kafka.clients.admin.MemberDescription;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class Coordinator extends Channel {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Coordinator.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  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.ofSeconds(1);
+
+  private final Catalog catalog;
+  private final IcebergSinkConfig config;
+  private final int totalPartitionCount;
+  private final String snapshotOffsetsProp;
+  private final ExecutorService exec;
+  private final CommitState commitState;
+
+  Coordinator(
+      Catalog catalog,
+      IcebergSinkConfig config,
+      Collection<MemberDescription> members,
+      KafkaClientFactory clientFactory,
+      SinkTaskContext context) {
+    // pass consumer group ID to which we commit low watermark offsets
+    super("coordinator", config.connectGroupId() + "-coord", config, 
clientFactory, context);
+
+    this.catalog = catalog;
+    this.config = config;
+    this.totalPartitionCount =
+        members.stream().mapToInt(desc -> 
desc.assignment().topicPartitions().size()).sum();
+    this.snapshotOffsetsProp =
+        String.format(
+            "kafka.connect.offsets.%s.%s", config.controlTopic(), 
config.connectGroupId());
+    this.exec = ThreadPools.newWorkerPool("iceberg-committer", 
config.commitThreads());
+    this.commitState = new CommitState(config);
+  }
+
+  void process() {
+    if (commitState.isCommitIntervalReached()) {
+      // send out begin commit
+      commitState.startNewCommit();
+      Event event =
+          new Event(config.connectGroupId(), new 
StartCommit(commitState.currentCommitId()));
+      send(event);
+      LOG.info("Commit {} initiated", commitState.currentCommitId());
+    }
+
+    consumeAvailable(POLL_DURATION);
+
+    if (commitState.isCommitTimedOut()) {
+      commit(true);
+    }
+  }
+
+  @Override
+  protected boolean receive(Envelope envelope) {
+    switch (envelope.event().payload().type()) {
+      case DATA_WRITTEN:
+        commitState.addResponse(envelope);
+        return true;
+      case DATA_COMPLETE:
+        commitState.addReady(envelope);
+        if (commitState.isCommitReady(totalPartitionCount)) {
+          commit(false);
+        }
+        return true;
+    }
+    return false;
+  }
+
+  private void commit(boolean partialCommit) {
+    try {
+      doCommit(partialCommit);
+    } catch (Exception e) {
+      LOG.warn("Commit failed, will try again next cycle", e);
+    } finally {
+      commitState.endCurrentCommit();
+    }
+  }
+
+  private void doCommit(boolean partialCommit) {
+    Map<TableReference, List<Envelope>> commitMap = 
commitState.tableCommitMap();
+
+    String offsetsJson = offsetsJson();
+    OffsetDateTime vtts = commitState.vtts(partialCommit);
+
+    Tasks.foreach(commitMap.entrySet())
+        .executeWith(exec)
+        .stopOnFailure()
+        .run(
+            entry -> {
+              commitToTable(entry.getKey(), entry.getValue(), offsetsJson, 
vtts);
+            });
+
+    // we should only get here if all tables committed successfully...
+    commitConsumerOffsets();
+    commitState.clearResponses();
+
+    Event event =
+        new Event(config.connectGroupId(), new 
CommitComplete(commitState.currentCommitId(), vtts));
+    send(event);
+
+    LOG.info(
+        "Commit {} complete, committed to {} table(s), vtts {}",
+        commitState.currentCommitId(),
+        commitMap.size(),
+        vtts);
+  }
+
+  private String offsetsJson() {
+    try {
+      return MAPPER.writeValueAsString(controlTopicOffsets());
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private void commitToTable(
+      TableReference tableReference,
+      List<Envelope> envelopeList,
+      String offsetsJson,
+      OffsetDateTime vtts) {
+    TableIdentifier tableIdentifier = tableReference.identifier();
+    Table table;
+    try {
+      table = catalog.loadTable(tableIdentifier);
+    } catch (NoSuchTableException e) {
+      LOG.warn("Table not found, skipping commit: {}", tableIdentifier, e);
+      return;
+    }
+
+    String branch = 
config.tableConfig(tableIdentifier.toString()).commitBranch();
+
+    Map<Integer, Long> committedOffsets = lastCommittedOffsetsForTable(table, 
branch);
+
+    List<DataWritten> payloads =
+        envelopeList.stream()
+            .filter(
+                envelope -> {
+                  Long minOffset = committedOffsets.get(envelope.partition());
+                  return minOffset == null || envelope.offset() >= minOffset;
+                })

Review Comment:
   Just a note for posterity's sake (not requesting any changes):
   While this is a good check, this does not guarantee _truly_ exactly-once. 
   There is an unlikely edge-case where if we have two coordinator zombies 
running, even this check will be insufficient to prevent duplicates. 
   Unless we add [support for conditional commits to 
iceberg](https://github.com/apache/iceberg/pull/6513), there's not much that 
can be done about it unfortunately (well, nothing simple anyway). 



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+class KafkaClientFactory {
+  private final Map<String, String> kafkaProps;
+
+  KafkaClientFactory(Map<String, String> kafkaProps) {
+    this.kafkaProps = kafkaProps;
+  }
+
+  Producer<String, byte[]> createProducer(String transactionalId) {
+    Map<String, Object> producerProps = Maps.newHashMap(kafkaProps);
+    producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
+    producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, 
UUID.randomUUID().toString());

Review Comment:
   ```suggestion
   ```
   
   Same here



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.data.Offset;
+import org.apache.iceberg.connect.data.SinkWriter;
+import org.apache.iceberg.connect.data.SinkWriterResult;
+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;
+import org.apache.iceberg.connect.events.StartCommit;
+import org.apache.iceberg.connect.events.TableReference;
+import org.apache.iceberg.connect.events.TopicPartitionOffset;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+
+class Worker extends Channel {
+
+  private final IcebergSinkConfig config;
+  private final SinkTaskContext context;
+  private final SinkWriter sinkWriter;
+
+  Worker(
+      IcebergSinkConfig config,
+      KafkaClientFactory clientFactory,
+      SinkWriter sinkWriter,
+      SinkTaskContext context) {
+    // pass transient consumer group ID to which we never commit offsets
+    super(
+        "worker",
+        IcebergSinkConfig.DEFAULT_CONTROL_GROUP_PREFIX + UUID.randomUUID(),
+        config,
+        clientFactory,
+        context);
+
+    this.config = config;
+    this.context = context;
+    this.sinkWriter = sinkWriter;
+  }
+
+  void process() {
+    consumeAvailable(Duration.ZERO);
+  }
+
+  @Override
+  protected boolean receive(Envelope envelope) {
+    Event event = envelope.event();
+    if (event.payload().type() != PayloadType.START_COMMIT) {
+      return false;
+    }
+
+    SinkWriterResult results = sinkWriter.completeWrite();
+
+    // include all assigned topic partitions even if no messages were read
+    // from a partition, as the coordinator will use that to determine
+    // when all data for a commit has been received
+    List<TopicPartitionOffset> assignments =
+        context.assignment().stream()
+            .map(
+                tp -> {
+                  Offset offset = results.sourceOffsets().get(tp);
+                  if (offset == null) {
+                    offset = Offset.NULL_OFFSET;
+                  }
+                  return new TopicPartitionOffset(
+                      tp.topic(), tp.partition(), offset.offset(), 
offset.timestamp());
+                })
+            .collect(Collectors.toList());
+
+    UUID commitId = ((StartCommit) event.payload()).commitId();
+
+    List<Event> events =
+        results.writerResults().stream()
+            .map(
+                writeResult ->
+                    new Event(
+                        config.connectGroupId(),
+                        new DataWritten(
+                            writeResult.partitionStruct(),
+                            commitId,
+                            TableReference.of(config.catalogName(), 
writeResult.tableIdentifier()),
+                            writeResult.dataFiles(),
+                            writeResult.deleteFiles())))
+            .collect(Collectors.toList());
+
+    Event readyEvent = new Event(config.connectGroupId(), new 
DataComplete(commitId, assignments));
+    events.add(readyEvent);
+
+    send(events, results.sourceOffsets());
+    context.requestCommit();

Review Comment:
   ```suggestion
   ```
   
   Why are we doing this at all? 
   The previous line already commits the offsets for the connector consumer 
group. 
   Plus, Kafka Connect runtime won't commit anything anyway since 
`IcebergSinkTask.preCommit` returns an empty map. 



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.util.concurrent.ExecutionException;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.ConsumerGroupDescription;
+import org.apache.kafka.clients.admin.DescribeConsumerGroupsResult;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+
+class KafkaUtils {
+
+  private static final String CONTEXT_CLASS_NAME =
+      "org.apache.kafka.connect.runtime.WorkerSinkTaskContext";
+
+  static ConsumerGroupDescription consumerGroupDescription(String 
consumerGroupId, Admin admin) {
+    try {
+      DescribeConsumerGroupsResult result =
+          admin.describeConsumerGroups(ImmutableList.of(consumerGroupId));
+      return result.describedGroups().get(consumerGroupId).get();
+
+    } catch (InterruptedException | ExecutionException e) {
+      throw new ConnectException(
+          "Cannot retrieve members for consumer group: " + consumerGroupId, e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  static ConsumerGroupMetadata consumerGroupMetadata(
+      SinkTaskContext context, String connectGroupId) {
+    if (CONTEXT_CLASS_NAME.equals(context.getClass().getName())) {
+      return ((Consumer<byte[], byte[]>)
+              DynFields.builder().hiddenImpl(CONTEXT_CLASS_NAME, 
"consumer").build(context).get())
+          .groupMetadata();
+    }
+    return new ConsumerGroupMetadata(connectGroupId);

Review Comment:
   Why are we defaulting to the non-zombie-fencing consumer-group-metadata? 
   IMO we should just hard fail, this shouldn't happen and if it does, we're 
risking duplicates. 
   



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+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.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.events.CommitComplete;
+import org.apache.iceberg.connect.events.CommitToTable;
+import org.apache.iceberg.connect.events.DataWritten;
+import org.apache.iceberg.connect.events.Event;
+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.Maps;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.kafka.clients.admin.MemberDescription;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class Coordinator extends Channel {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Coordinator.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  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.ofSeconds(1);
+
+  private final Catalog catalog;
+  private final IcebergSinkConfig config;
+  private final int totalPartitionCount;
+  private final String snapshotOffsetsProp;
+  private final ExecutorService exec;
+  private final CommitState commitState;
+
+  Coordinator(
+      Catalog catalog,
+      IcebergSinkConfig config,
+      Collection<MemberDescription> members,
+      KafkaClientFactory clientFactory,
+      SinkTaskContext context) {
+    // pass consumer group ID to which we commit low watermark offsets
+    super("coordinator", config.connectGroupId() + "-coord", config, 
clientFactory, context);
+
+    this.catalog = catalog;
+    this.config = config;
+    this.totalPartitionCount =
+        members.stream().mapToInt(desc -> 
desc.assignment().topicPartitions().size()).sum();
+    this.snapshotOffsetsProp =
+        String.format(
+            "kafka.connect.offsets.%s.%s", config.controlTopic(), 
config.connectGroupId());
+    this.exec = ThreadPools.newWorkerPool("iceberg-committer", 
config.commitThreads());
+    this.commitState = new CommitState(config);
+  }
+
+  void process() {
+    if (commitState.isCommitIntervalReached()) {
+      // send out begin commit
+      commitState.startNewCommit();
+      Event event =
+          new Event(config.connectGroupId(), new 
StartCommit(commitState.currentCommitId()));
+      send(event);
+      LOG.info("Commit {} initiated", commitState.currentCommitId());
+    }
+
+    consumeAvailable(POLL_DURATION);
+
+    if (commitState.isCommitTimedOut()) {
+      commit(true);
+    }
+  }
+
+  @Override
+  protected boolean receive(Envelope envelope) {
+    switch (envelope.event().payload().type()) {
+      case DATA_WRITTEN:
+        commitState.addResponse(envelope);
+        return true;
+      case DATA_COMPLETE:
+        commitState.addReady(envelope);
+        if (commitState.isCommitReady(totalPartitionCount)) {
+          commit(false);
+        }
+        return true;
+    }
+    return false;
+  }
+
+  private void commit(boolean partialCommit) {
+    try {
+      doCommit(partialCommit);
+    } catch (Exception e) {
+      LOG.warn("Commit failed, will try again next cycle", e);
+    } finally {
+      commitState.endCurrentCommit();
+    }
+  }
+
+  private void doCommit(boolean partialCommit) {
+    Map<TableReference, List<Envelope>> commitMap = 
commitState.tableCommitMap();
+
+    String offsetsJson = offsetsJson();
+    OffsetDateTime vtts = commitState.vtts(partialCommit);
+
+    Tasks.foreach(commitMap.entrySet())
+        .executeWith(exec)
+        .stopOnFailure()
+        .run(
+            entry -> {
+              commitToTable(entry.getKey(), entry.getValue(), offsetsJson, 
vtts);
+            });
+
+    // we should only get here if all tables committed successfully...
+    commitConsumerOffsets();
+    commitState.clearResponses();
+
+    Event event =
+        new Event(config.connectGroupId(), new 
CommitComplete(commitState.currentCommitId(), vtts));
+    send(event);
+
+    LOG.info(
+        "Commit {} complete, committed to {} table(s), vtts {}",
+        commitState.currentCommitId(),
+        commitMap.size(),
+        vtts);
+  }
+
+  private String offsetsJson() {
+    try {
+      return MAPPER.writeValueAsString(controlTopicOffsets());
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private void commitToTable(
+      TableReference tableReference,
+      List<Envelope> envelopeList,
+      String offsetsJson,
+      OffsetDateTime vtts) {
+    TableIdentifier tableIdentifier = tableReference.identifier();
+    Table table;
+    try {
+      table = catalog.loadTable(tableIdentifier);
+    } catch (NoSuchTableException e) {
+      LOG.warn("Table not found, skipping commit: {}", tableIdentifier, e);
+      return;
+    }
+
+    String branch = 
config.tableConfig(tableIdentifier.toString()).commitBranch();
+
+    Map<Integer, Long> committedOffsets = lastCommittedOffsetsForTable(table, 
branch);
+
+    List<DataWritten> payloads =
+        envelopeList.stream()
+            .filter(
+                envelope -> {
+                  Long minOffset = committedOffsets.get(envelope.partition());
+                  return minOffset == null || envelope.offset() >= minOffset;
+                })
+            .map(envelope -> (DataWritten) envelope.event().payload())
+            .collect(Collectors.toList());
+
+    List<DataFile> dataFiles =
+        payloads.stream()
+            .filter(payload -> payload.dataFiles() != null)
+            .flatMap(payload -> payload.dataFiles().stream())
+            .filter(dataFile -> dataFile.recordCount() > 0)
+            .filter(distinctByKey(dataFile -> dataFile.path().toString()))
+            .collect(Collectors.toList());
+
+    List<DeleteFile> deleteFiles =
+        payloads.stream()
+            .filter(payload -> payload.deleteFiles() != null)
+            .flatMap(payload -> payload.deleteFiles().stream())
+            .filter(deleteFile -> deleteFile.recordCount() > 0)
+            .filter(distinctByKey(deleteFile -> deleteFile.path().toString()))
+            .collect(Collectors.toList());
+
+    if (dataFiles.isEmpty() && deleteFiles.isEmpty()) {
+      LOG.info("Nothing to commit to table {}, skipping", tableIdentifier);
+    } else {
+      if (deleteFiles.isEmpty()) {
+        AppendFiles appendOp = table.newAppend();
+        if (branch != null) {
+          appendOp.toBranch(branch);
+        }
+        appendOp.set(snapshotOffsetsProp, offsetsJson);
+        appendOp.set(COMMIT_ID_SNAPSHOT_PROP, 
commitState.currentCommitId().toString());
+        if (vtts != null) {
+          appendOp.set(VTTS_SNAPSHOT_PROP, vtts.toString());
+        }
+        dataFiles.forEach(appendOp::appendFile);
+        appendOp.commit();

Review Comment:
   I think you forgot to port over the the code for handling multiple 
partition-specs 
   
https://github.com/tabular-io/iceberg-kafka-connect/commit/cdd54f3851b2b255702716008d6ab97cfbf97013
   
   Otherwise, connectors can become hard-stuck when users evolve the partition 
whilst the connector is running, which I don't think is an entirely 
unreasonable expectation. 



##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.iceberg.connect.channel;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.iceberg.connect.IcebergSinkConfig;
+import org.apache.iceberg.connect.data.Offset;
+import org.apache.iceberg.connect.data.SinkWriter;
+import org.apache.iceberg.connect.data.SinkWriterResult;
+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;
+import org.apache.iceberg.connect.events.StartCommit;
+import org.apache.iceberg.connect.events.TableReference;
+import org.apache.iceberg.connect.events.TopicPartitionOffset;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+
+class Worker extends Channel {
+
+  private final IcebergSinkConfig config;
+  private final SinkTaskContext context;
+  private final SinkWriter sinkWriter;
+
+  Worker(
+      IcebergSinkConfig config,
+      KafkaClientFactory clientFactory,
+      SinkWriter sinkWriter,
+      SinkTaskContext context) {
+    // pass transient consumer group ID to which we never commit offsets
+    super(
+        "worker",
+        IcebergSinkConfig.DEFAULT_CONTROL_GROUP_PREFIX + UUID.randomUUID(),
+        config,
+        clientFactory,
+        context);
+
+    this.config = config;
+    this.context = context;
+    this.sinkWriter = sinkWriter;
+  }
+
+  void process() {
+    consumeAvailable(Duration.ZERO);
+  }
+
+  @Override
+  protected boolean receive(Envelope envelope) {
+    Event event = envelope.event();
+    if (event.payload().type() != PayloadType.START_COMMIT) {
+      return false;
+    }
+
+    SinkWriterResult results = sinkWriter.completeWrite();
+
+    // include all assigned topic partitions even if no messages were read
+    // from a partition, as the coordinator will use that to determine
+    // when all data for a commit has been received
+    List<TopicPartitionOffset> assignments =
+        context.assignment().stream()
+            .map(
+                tp -> {
+                  Offset offset = results.sourceOffsets().get(tp);
+                  if (offset == null) {
+                    offset = Offset.NULL_OFFSET;
+                  }
+                  return new TopicPartitionOffset(
+                      tp.topic(), tp.partition(), offset.offset(), 
offset.timestamp());
+                })
+            .collect(Collectors.toList());
+
+    UUID commitId = ((StartCommit) event.payload()).commitId();
+
+    List<Event> events =
+        results.writerResults().stream()
+            .map(
+                writeResult ->
+                    new Event(
+                        config.connectGroupId(),
+                        new DataWritten(
+                            writeResult.partitionStruct(),
+                            commitId,
+                            TableReference.of(config.catalogName(), 
writeResult.tableIdentifier()),
+                            writeResult.dataFiles(),
+                            writeResult.deleteFiles())))
+            .collect(Collectors.toList());
+
+    Event readyEvent = new Event(config.connectGroupId(), new 
DataComplete(commitId, assignments));
+    events.add(readyEvent);
+
+    send(events, results.sourceOffsets());
+    context.requestCommit();

Review Comment:
   abc



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

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

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


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

Reply via email to