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 @@ -92,6 +92,7 @@ public class IcebergSinkConfig extends AbstractConfig {
private static final String HADOOP_CONF_DIR_PROP = "iceberg.hadoop-conf-dir";

private static final String NAME_PROP = "name";
private static final String TASK_ID = "task.id";
private static final String BOOTSTRAP_SERVERS_PROP = "bootstrap.servers";

private static final String DEFAULT_CATALOG_NAME = "iceberg";
Expand All @@ -101,6 +102,9 @@ public class IcebergSinkConfig extends AbstractConfig {
public static final int SCHEMA_UPDATE_RETRIES = 2; // 3 total attempts
public static final int CREATE_TABLE_RETRIES = 2; // 3 total attempts

private static final String COORDINATOR_EXECUTOR_KEEP_ALIVE_TIMEOUT_MS =
"iceberg.coordinator-executor-keep-alive-timeout-ms";

@VisibleForTesting static final String COMMA_NO_PARENS_REGEX = ",(?![^()]*+\\))";

public static final ConfigDef CONFIG_DEF = newConfigDef();
Expand Down Expand Up @@ -225,6 +229,12 @@ private static ConfigDef newConfigDef() {
null,
Importance.MEDIUM,
"If specified, Hadoop config files in this directory will be loaded");
configDef.define(
COORDINATOR_EXECUTOR_KEEP_ALIVE_TIMEOUT_MS,
ConfigDef.Type.LONG,
120000L,
Importance.LOW,
"config to control coordinator executor keep alive time");
return configDef;
}

Expand Down Expand Up @@ -284,6 +294,10 @@ public String connectorName() {
return originalProps.get(NAME_PROP);
}

public String taskId() {
return originalProps.get(TASK_ID);
}

public String transactionalSuffix() {
// this is for internal use and is not part of the config definition...
return originalProps.get(INTERNAL_TRANSACTIONAL_SUFFIX_PROP);
Expand Down Expand Up @@ -337,6 +351,10 @@ public String tablesDefaultPartitionBy() {
return getString(TABLES_DEFAULT_PARTITION_BY);
}

public long keepAliveTimeoutInMs() {
return getLong(COORDINATOR_EXECUTOR_KEEP_ALIVE_TIMEOUT_MS);
}

public TableSinkConfig tableConfig(String tableName) {
return tableConfigMap.computeIfAbsent(
tableName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ public List<Map<String, String>> taskConfigs(int maxTasks) {
i -> {
Map<String, String> map = Maps.newHashMap(props);
map.put(IcebergSinkConfig.INTERNAL_TRANSACTIONAL_SUFFIX_PROP, txnSuffix + i);
map.put("task.id", String.valueOf(i));
return map;
})
.collect(Collectors.toList());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,11 +136,23 @@ public void stop() {

@Override
public void close(Collection<TopicPartition> closedPartitions) {
if (!isInitialized.get()) {
LOG.warn("Unexpected close() call without resource initialization");
return;
}
if (hasLeaderPartition(closedPartitions)) {
LOG.info("Committer lost leader partition. Stopping Coordinator.");
LOG.info(
"Committer {}-{} lost leader partition. Stopping Coordinator.",
config.connectorName(),
config.taskId());
stopCoordinator();
}
LOG.info("Stopping worker {}-{}.", config.connectorName(), config.taskId());
stopWorker();
LOG.info(
"Seeking to last committed offsets for worker {}-{}.",
config.connectorName(),
config.taskId());
KafkaUtils.seekToLastCommittedOffsets(context);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.function.Predicate;
Expand All @@ -51,8 +53,8 @@
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.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.iceberg.util.Tasks;
import org.apache.iceberg.util.ThreadPools;
import org.apache.kafka.clients.admin.MemberDescription;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.sink.SinkTaskContext;
Expand All @@ -73,7 +75,6 @@ class Coordinator extends Channel {
private final String snapshotOffsetsProp;
private final ExecutorService exec;
private final CommitState commitState;
private volatile boolean terminated;

Coordinator(
Catalog catalog,
Expand All @@ -91,7 +92,17 @@ class Coordinator extends Channel {
this.snapshotOffsetsProp =
String.format(
"kafka.connect.offsets.%s.%s", config.controlTopic(), config.connectGroupId());
this.exec = ThreadPools.newFixedThreadPool("iceberg-committer", config.commitThreads());
this.exec =
new ThreadPoolExecutor(
config.commitThreads(),
config.commitThreads(),
config.keepAliveTimeoutInMs(),
TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<>(),
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("iceberg-committer" + "-%d")
.build());
this.commitState = new CommitState(config);
}

Expand Down Expand Up @@ -221,10 +232,6 @@ 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 @@ -304,10 +311,7 @@ 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 @@ -317,4 +321,10 @@ 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,6 +64,5 @@ boolean isTerminated() {

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