danielcweeks commented on code in PR #10351: URL: https://github.com/apache/iceberg/pull/10351#discussion_r1672743681
########## kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.java: ########## @@ -0,0 +1,29 @@ +/* + * 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; + +import org.apache.iceberg.connect.channel.CommitterImpl; + +public class CommitterFactory { + public static Committer createCommitter(IcebergSinkConfig config) { Review Comment: nit: I don't think this needs to be public ########## 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)); + } + producer.commitTransaction(); + } catch (Exception e) { + try { + producer.abortTransaction(); + } catch (Exception ex) { + LOG.warn("Error aborting producer transaction", ex); + } + throw e; + } + } + } + + protected abstract boolean receive(Envelope envelope); + + protected void consumeAvailable(Duration pollDuration) { + ConsumerRecords<String, byte[]> records = consumer.poll(pollDuration); + while (!records.isEmpty()) { + records.forEach( + record -> { + // 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); + + Event event = AvroUtil.decode(record.value()); + + if (event.groupId().equals(connectGroupId)) { + LOG.debug("Received event of type: {}", event.type().name()); + if (receive(new Envelope(event, record.partition(), record.offset()))) { + LOG.info("Handled event of type: {}", event.type().name()); + } + } + }); + records = consumer.poll(pollDuration); + } + } + + protected Map<Integer, Long> controlTopicOffsets() { + return controlTopicOffsets; + } + + protected void commitConsumerOffsets() { + Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = Maps.newHashMap(); + controlTopicOffsets() + .forEach( + (k, v) -> + offsetsToCommit.put(new TopicPartition(controlTopic, k), new OffsetAndMetadata(v))); + consumer.commitSync(offsetsToCommit); + } + + protected Admin admin() { Review Comment: No usages for this, is it necessary? ########## kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java: ########## @@ -0,0 +1,311 @@ +/* + * 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.concurrent.TimeUnit; +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 {}", Review Comment: `vtts` -> `valid through {}` ########## 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. Commit ID: {}", currentCommitId); + return true; + } + return false; + } + + boolean isCommitReady(int expectedPartitionCount) { + if (!isCommitInProgress()) { + return false; + } + + int receivedPartitionCount = + readyBuffer.stream() + .filter(payload -> payload.commitId().equals(currentCommitId)) + .mapToInt(payload -> payload.assignments().size()) + .sum(); + + if (receivedPartitionCount >= expectedPartitionCount) { + LOG.info( + "Commit {} ready, received responses for all {} partitions", + currentCommitId, + receivedPartitionCount); + return true; + } + + LOG.info( + "Commit {} not ready, received responses for {} of {} partitions, waiting for more", + currentCommitId, + receivedPartitionCount, + expectedPartitionCount); + + return false; + } + + Map<TableReference, List<Envelope>> tableCommitMap() { + return commitBuffer.stream() + .collect( + Collectors.groupingBy( + envelope -> ((DataWritten) envelope.event().payload()).tableReference())); + } + + OffsetDateTime vtts(boolean partialCommit) { Review Comment: We probably to avoid acronyms like `vtts` as they're not universally understood. `validThroughTs`? ########## kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java: ########## @@ -0,0 +1,311 @@ +/* + * 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.concurrent.TimeUnit; +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"; Review Comment: Same comment about VTTS here, use fully expanded names: VALID_THROUGH_TS ########## 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. Commit ID: {}", currentCommitId); + return true; + } + return false; + } + + boolean isCommitReady(int expectedPartitionCount) { + if (!isCommitInProgress()) { + return false; + } + + int receivedPartitionCount = + readyBuffer.stream() + .filter(payload -> payload.commitId().equals(currentCommitId)) + .mapToInt(payload -> payload.assignments().size()) + .sum(); + + if (receivedPartitionCount >= expectedPartitionCount) { + LOG.info( + "Commit {} ready, received responses for all {} partitions", + currentCommitId, + receivedPartitionCount); + return true; + } + + LOG.info( + "Commit {} not ready, received responses for {} of {} partitions, waiting for more", + currentCommitId, + receivedPartitionCount, + expectedPartitionCount); + + return false; + } + + Map<TableReference, List<Envelope>> tableCommitMap() { + return commitBuffer.stream() + .collect( + Collectors.groupingBy( + envelope -> ((DataWritten) envelope.event().payload()).tableReference())); + } + + OffsetDateTime vtts(boolean partialCommit) { + boolean validVtts = + !partialCommit + && readyBuffer.stream() + .flatMap(event -> event.assignments().stream()) + .allMatch(offset -> offset.timestamp() != null); + + OffsetDateTime result; + if (validVtts) { + result = + readyBuffer.stream() + .flatMap(event -> event.assignments().stream()) + .map(TopicPartitionOffset::timestamp) + .min(Comparator.naturalOrder()) + .get(); Review Comment: Are we sure this `get()` is safe? The `allMatch` above will return ture if there are no elements. -- 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