Skip to content
Merged
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 @@ -19,6 +19,7 @@
package org.apache.iceberg.connect;

import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -61,7 +62,7 @@ public void close(Collection<TopicPartition> partitions) {

private void close() {
if (committer != null) {
committer.close(context.assignment());
committer.close(List.of());
committer = null;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,19 +136,32 @@ public void stop() {

@Override
public void close(Collection<TopicPartition> closedPartitions) {
// Always try to stop the worker to avoid duplicates.
stopWorker();

// Defensive: close called without prior initialization (should not happen).
if (!isInitialized.get()) {
LOG.warn("Unexpected close() call without resource initialization");
LOG.warn("Close unexpectedly called without partition assignment");
return;
}

// Empty partitions → task was stopped explicitly. Stop coordinator if running.
if (closedPartitions.isEmpty()) {
LOG.info("Task stopped. Closing coordinator.");
stopCoordinator();
return;
}

// Normal close: if leader partition is lost, stop coordinator.
if (hasLeaderPartition(closedPartitions)) {
LOG.info(
"Committer {}-{} lost leader partition. Stopping Coordinator.",
"Committer {}-{} lost leader partition. Stopping coordinator.",
config.connectorName(),
config.taskId());
stopCoordinator();
}
LOG.info("Stopping worker {}-{}.", config.connectorName(), config.taskId());
stopWorker();

// Reset offsets to last committed to avoid data loss.
LOG.info(
"Seeking to last committed offsets for worker {}-{}.",
config.connectorName(),
Expand All @@ -167,7 +180,10 @@ public void save(Collection<SinkRecord> sinkRecords) {

private void processControlEvents() {
if (coordinatorThread != null && coordinatorThread.isTerminated()) {
throw new NotRunningException("Coordinator unexpectedly terminated");
throw new NotRunningException(
String.format(
"Coordinator unexpectedly terminated on committer %s-%s",
config.connectorName(), config.taskId()));
}
if (worker != null) {
worker.process();
Expand All @@ -176,7 +192,7 @@ private void processControlEvents() {

private void startWorker() {
if (null == this.worker) {
LOG.info("Starting commit worker");
LOG.info("Starting commit worker {}-{}", config.connectorName(), config.taskId());
SinkWriter sinkWriter = new SinkWriter(catalog, config);
worker = new Worker(config, clientFactory, sinkWriter, context);
worker.start();
Expand All @@ -185,7 +201,10 @@ private void startWorker() {

private void startCoordinator() {
if (null == this.coordinatorThread) {
LOG.info("Task elected leader, starting commit coordinator");
LOG.info(
"Task {}-{} elected leader, starting commit coordinator",
config.connectorName(),
config.taskId());
Coordinator coordinator =
new Coordinator(catalog, config, membersWhenWorkerIsCoordinator, clientFactory, context);
coordinatorThread = new CoordinatorThread(coordinator);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ class Coordinator extends Channel {
private final String snapshotOffsetsProp;
private final ExecutorService exec;
private final CommitState commitState;
private volatile boolean terminated;

Coordinator(
Catalog catalog,
Expand Down Expand Up @@ -232,6 +233,10 @@ private void commitToTable(
.filter(distinctByKey(ContentFile::location))
.collect(Collectors.toList());

if (terminated) {
throw new ConnectException("Coordinator is terminated, commit aborted");
}

if (dataFiles.isEmpty() && deleteFiles.isEmpty()) {
LOG.info("Nothing to commit to table {}, skipping", tableIdentifier);
} else {
Expand Down Expand Up @@ -311,7 +316,10 @@ private Map<Integer, Long> lastCommittedOffsetsForTable(Table table, String bran
}

void terminate() {
this.terminated = true;

exec.shutdownNow();

// wait for coordinator termination, else cause the sink task to fail
try {
if (!exec.awaitTermination(1, TimeUnit.MINUTES)) {
Expand All @@ -321,10 +329,4 @@ void terminate() {
throw new ConnectException("Interrupted while waiting for coordinator shutdown", e);
}
}

@Override
public void stop() {
terminate();
super.stop();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,5 +64,6 @@ boolean isTerminated() {

void terminate() {
this.terminated = true;
coordinator.terminate();
}
}