diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index c986f8afc2eb..eec133acc683 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -150,11 +150,10 @@ protected boolean receive(Envelope envelope) { private void commit(boolean partialCommit) { try { doCommit(partialCommit); - } catch (Exception e) { - LOG.warn( - "Coordinator {} failed to commit for commit {}, will try again next cycle", - taskId, - commitState.currentCommitId(), + } catch (RuntimeException e) { + throw new RuntimeException( + String.format( + "Coordinator %s failed to commit %s", taskId, commitState.currentCommitId()), e); } finally { commitState.endCurrentCommit(); diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java index ed370fcdad35..cea608c2edb1 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java @@ -19,12 +19,16 @@ package org.apache.iceberg.connect.channel; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import java.time.OffsetDateTime; import java.util.List; import java.util.UUID; +import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DataOperations; @@ -45,6 +49,8 @@ import org.apache.iceberg.connect.events.StartCommit; import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types.StructType; @@ -135,7 +141,11 @@ public void testCommitError() { .withRecordCount(5) .build(); - coordinatorTest(ImmutableList.of(badDataFile), ImmutableList.of(), null); + assertThatThrownBy( + () -> coordinatorTest(ImmutableList.of(badDataFile), ImmutableList.of(), null)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("failed to commit") + .hasRootCauseInstanceOf(IllegalArgumentException.class); // no commit messages sent assertThat(producer.history()).hasSize(1); @@ -143,6 +153,31 @@ public void testCommitError() { assertThat(table.snapshots()).isEmpty(); } + @Test + public void testCommitFailedExceptionPropagates() { + // Reproduce issue #15878: a CommitFailedException from the catalog (e.g., Glue concurrent + // update) must propagate out of Coordinator.process() so that CoordinatorThread terminates + // and the Kafka Connect task transitions to FAILED instead of silently dropping data. + Table spiedTable = spy(table); + AppendFiles spiedAppend = spy(table.newAppend()); + doThrow(new CommitFailedException("Glue detected concurrent update")) + .when(spiedAppend) + .commit(); + when(spiedTable.newAppend()).thenReturn(spiedAppend); + when(catalog.loadTable(TABLE_IDENTIFIER)).thenReturn(spiedTable); + + assertThatThrownBy( + () -> + coordinatorTest( + ImmutableList.of(EventTestUtil.createDataFile()), + ImmutableList.of(), + EventTestUtil.now())) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("failed to commit") + .hasRootCauseInstanceOf(CommitFailedException.class) + .hasRootCauseMessage("Glue detected concurrent update"); + } + private void assertCommitTable(int idx, UUID commitId, OffsetDateTime ts) { byte[] bytes = producer.history().get(idx).value(); Event commitTable = AvroUtil.decode(bytes); @@ -289,9 +324,17 @@ public void testCoordinatorCommittedOffsetValidation() { Snapshot firstSnapshot = table.currentSnapshot(); assertThat(firstSnapshot.summary()).containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":7}"); - // Trigger commit to the table - coordinatorTest( - ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), EventTestUtil.now()); + // Trigger commit to the table; the coordinator detects stale offsets and surfaces the + // ValidationException instead of silently swallowing it (see issue #15878). + assertThatThrownBy( + () -> + coordinatorTest( + ImmutableList.of(EventTestUtil.createDataFile()), + ImmutableList.of(), + EventTestUtil.now())) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("failed to commit") + .hasRootCauseInstanceOf(ValidationException.class); // Assert that the table was not updated and offsets remain table.refresh();