Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -148,14 +148,18 @@ protected boolean receive(Envelope envelope) {
}

private void commit(boolean partialCommit) {
// Do not swallow commit failures: propagate them so CoordinatorThread terminates
// and the Kafka Connect task transitions to FAILED instead of silently dropping data
// (e.g., CommitFailedException from catalogs that detect concurrent updates).
try {
doCommit(partialCommit);
} catch (Exception e) {
LOG.warn(
"Coordinator {} failed to commit for commit {}, will try again next cycle",
} catch (RuntimeException e) {
LOG.error(
"Coordinator {} failed to commit for commit {}; propagating failure to terminate task",
taskId,
commitState.currentCommitId(),
e);
throw e;
Comment on lines +157 to +162
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

change it to

    throw new RuntimeException(
          String.format("Coordinator %s failed to commit %s",
              taskId, commitState.currentCommitId()),
          e);

This allows the further up CoordinatorThread.run() catch to log the error once, and still attribute the error to this location.

} finally {
commitState.endCurrentCommit();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -135,14 +141,40 @@ public void testCommitError() {
.withRecordCount(5)
.build();

coordinatorTest(ImmutableList.of(badDataFile), ImmutableList.of(), null);
assertThatThrownBy(
() -> coordinatorTest(ImmutableList.of(badDataFile), ImmutableList.of(), null))
.isInstanceOf(IllegalArgumentException.class)
.hasMessageContaining("Cannot find partition spec");

// no commit messages sent
assertThat(producer.history()).hasSize(1);

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(CommitFailedException.class)
.hasMessageContaining("Glue detected concurrent update");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if you do the above change, then i think this need to be :

        .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);
Expand Down Expand Up @@ -289,9 +321,16 @@ 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(ValidationException.class)
.hasMessageContaining("stale offsets");

// Assert that the table was not updated and offsets remain
table.refresh();
Expand Down
Loading