Skip to content

Commit

Permalink
Refactor into Writer and Committer
Browse files Browse the repository at this point in the history
  • Loading branch information
fqaiser94 committed Apr 17, 2024
1 parent 423b4a8 commit 375bc15
Show file tree
Hide file tree
Showing 41 changed files with 2,198 additions and 829 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ private void runTest(String branch, boolean useSchema) {
.config("value.converter.schemas.enable", useSchema)
.config("iceberg.tables", String.format("%s.%s", TEST_DB, TEST_TABLE))
.config("iceberg.tables.cdc-field", "op")
.config("iceberg.control.commit.interval-ms", 1000)
.config("iceberg.control.commit.interval-ms", 5000)
.config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE)
.config("iceberg.kafka.auto.offset.reset", "earliest");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ private void runTest(String branch, boolean useSchema) {
.config("value.converter.schemas.enable", useSchema)
.config("iceberg.tables.dynamic-enabled", true)
.config("iceberg.tables.route-field", "payload")
.config("iceberg.control.commit.interval-ms", 1000)
.config("iceberg.control.commit.interval-ms", 5000)
.config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE)
.config("iceberg.kafka.auto.offset.reset", "earliest");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ private void runTest(String branch, boolean useSchema) {
.config("iceberg.tables.route-field", "type")
.config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE1), "type1")
.config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE2), "type2")
.config("iceberg.control.commit.interval-ms", 1000)
.config("iceberg.control.commit.interval-ms", 5000)
.config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE)
.config("iceberg.kafka.auto.offset.reset", "earliest");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -229,7 +229,7 @@ private void runTest(String branch, boolean useSchema, Map<String, String> extra
.config("value.converter", "org.apache.kafka.connect.json.JsonConverter")
.config("value.converter.schemas.enable", useSchema)
.config("iceberg.tables", String.format("%s.%s", TEST_DB, TEST_TABLE))
.config("iceberg.control.commit.interval-ms", 1000)
.config("iceberg.control.commit.interval-ms", 5000)
.config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE)
.config("iceberg.kafka.auto.offset.reset", "earliest");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,7 @@ public class IcebergSinkConfig extends AbstractConfig {

private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkConfig.class.getName());

public static final String INTERNAL_TRANSACTIONAL_SUFFIX_PROP =
"iceberg.coordinator.transactional.suffix";
public static final String INTERNAL_TASK_ID = "kafka.connect.task.id";
private static final String ROUTE_REGEX = "route-regex";
private static final String ID_COLUMNS = "id-columns";
private static final String PARTITION_BY = "partition-by";
Expand Down Expand Up @@ -296,9 +295,9 @@ public String connectorName() {
return originalProps.get(NAME_PROP);
}

public String transactionalSuffix() {
public Integer taskId() {
// this is for internal use and is not part of the config definition...
return originalProps.get(INTERNAL_TRANSACTIONAL_SUFFIX_PROP);
return Integer.valueOf(originalProps.get(INTERNAL_TASK_ID));
}

public Map<String, String> catalogProps() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@

import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.stream.IntStream;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.kafka.common.config.ConfigDef;
Expand Down Expand Up @@ -50,12 +49,11 @@ public Class<? extends Task> taskClass() {

@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
String txnSuffix = "-txn-" + UUID.randomUUID() + "-";
return IntStream.range(0, maxTasks)
.mapToObj(
i -> {
Map<String, String> map = Maps.newHashMap(props);
map.put(IcebergSinkConfig.INTERNAL_TRANSACTIONAL_SUFFIX_PROP, txnSuffix + i);
map.put(IcebergSinkConfig.INTERNAL_TASK_ID, Integer.toString(i));
return map;
})
.collect(toList());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,27 +18,16 @@
*/
package io.tabular.iceberg.connect;

import io.tabular.iceberg.connect.channel.Coordinator;
import io.tabular.iceberg.connect.channel.CoordinatorThread;
import io.tabular.iceberg.connect.channel.KafkaClientFactory;
import io.tabular.iceberg.connect.channel.KafkaUtils;
import io.tabular.iceberg.connect.channel.NotRunningException;
import io.tabular.iceberg.connect.channel.Worker;
import io.tabular.iceberg.connect.data.IcebergWriterFactory;
import io.tabular.iceberg.connect.api.Committer;
import io.tabular.iceberg.connect.api.Writer;
import io.tabular.iceberg.connect.channel.CommitterImpl;
import io.tabular.iceberg.connect.data.Utilities;
import io.tabular.iceberg.connect.data.WriterImpl;
import java.util.Collection;
import java.util.Comparator;
import java.util.Map;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.ConsumerGroupDescription;
import org.apache.kafka.clients.admin.MemberDescription;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.ConsumerGroupState;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import org.slf4j.Logger;
Expand All @@ -49,21 +38,8 @@ public class IcebergSinkTask extends SinkTask {
private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkTask.class);

private IcebergSinkConfig config;
private Catalog catalog;
private CoordinatorThread coordinatorThread;
private Worker worker;

static class TopicPartitionComparator implements Comparator<TopicPartition> {

@Override
public int compare(TopicPartition o1, TopicPartition o2) {
int result = o1.topic().compareTo(o2.topic());
if (result == 0) {
result = Integer.compare(o1.partition(), o2.partition());
}
return result;
}
}
private Writer writer;
private Committer committer;

@Override
public String version() {
Expand All @@ -75,114 +51,44 @@ public void start(Map<String, String> props) {
this.config = new IcebergSinkConfig(props);
}

@Override
public void open(Collection<TopicPartition> partitions) {
// destroy any state if KC re-uses object
clearObjectState();
catalog = Utilities.loadCatalog(config);
KafkaClientFactory clientFactory = new KafkaClientFactory(config.kafkaProps());

ConsumerGroupDescription groupDesc;
try (Admin admin = clientFactory.createAdmin()) {
groupDesc = KafkaUtils.consumerGroupDescription(config.connectGroupId(), admin);
}

if (groupDesc.state() == ConsumerGroupState.STABLE) {
Collection<MemberDescription> members = groupDesc.members();
if (isLeader(members, partitions)) {
LOG.info("Task elected leader, starting commit coordinator");
Coordinator coordinator = new Coordinator(catalog, config, members, clientFactory);
coordinatorThread = new CoordinatorThread(coordinator);
coordinatorThread.start();
}
}

LOG.info("Starting commit worker");
IcebergWriterFactory writerFactory = new IcebergWriterFactory(catalog, config);
worker = new Worker(config, clientFactory, writerFactory, context);
worker.syncCommitOffsets();
worker.start();
}

@VisibleForTesting
boolean isLeader(Collection<MemberDescription> members, Collection<TopicPartition> partitions) {
// there should only be one task assigned partition 0 of the first topic,
// so elect that one the leader
TopicPartition firstTopicPartition =
members.stream()
.flatMap(member -> member.assignment().topicPartitions().stream())
.min(new TopicPartitionComparator())
.orElseThrow(
() -> new ConnectException("No partitions assigned, cannot determine leader"));
private void clearState() {
Utilities.close(writer);
writer = null;

return partitions.contains(firstTopicPartition);
Utilities.close(committer);
committer = null;
}

@Override
public void close(Collection<TopicPartition> partitions) {
close();
}

private void close() {
clearObjectState();
}

private void clearObjectState() {
if (worker != null) {
worker.stop();
worker = null;
}

if (coordinatorThread != null) {
coordinatorThread.terminate();
coordinatorThread = null;
}
public void open(Collection<TopicPartition> partitions) {
// destroy any state if KC re-uses object
clearState();

if (catalog != null) {
if (catalog instanceof AutoCloseable) {
try {
((AutoCloseable) catalog).close();
} catch (Exception e) {
LOG.warn("An error occurred closing catalog instance, ignoring...", e);
}
}
catalog = null;
}
this.writer = new WriterImpl(config);
this.committer = new CommitterImpl(context, config);
}

@Override
public void put(Collection<SinkRecord> sinkRecords) {
if (sinkRecords != null && !sinkRecords.isEmpty() && worker != null) {
worker.save(sinkRecords);
if (writer != null && committer != null) {
writer.write(sinkRecords);
committer.commit(writer);
}
processControlEvents();
}

@Override
public void flush(Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
processControlEvents();
}

private void processControlEvents() {
if (coordinatorThread != null && coordinatorThread.isTerminated()) {
throw new NotRunningException("Coordinator unexpectedly terminated");
}
if (worker != null) {
worker.process();
}
public Map<TopicPartition, OffsetAndMetadata> preCommit(
Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
return ImmutableMap.of();
}

@Override
public Map<TopicPartition, OffsetAndMetadata> preCommit(
Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
if (worker == null) {
return ImmutableMap.of();
}
return worker.commitOffsets();
public void close(Collection<TopicPartition> partitions) {
clearState();
}

@Override
public void stop() {
close();
clearState();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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 io.tabular.iceberg.connect.api;

import io.tabular.iceberg.connect.data.Offset;
import io.tabular.iceberg.connect.data.WriterResult;
import java.util.List;
import java.util.Map;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.kafka.common.TopicPartition;

public class Committable {

private final ImmutableMap<TopicPartition, Offset> offsetsByTopicPartition;
private final ImmutableList<WriterResult> writerResults;

public Committable(
Map<TopicPartition, Offset> offsetsByTopicPartition, List<WriterResult> writerResults) {
this.offsetsByTopicPartition = ImmutableMap.copyOf(offsetsByTopicPartition);
this.writerResults = ImmutableList.copyOf(writerResults);
}

public Map<TopicPartition, Offset> offsetsByTopicPartition() {
return offsetsByTopicPartition;
}

public List<WriterResult> writerResults() {
return writerResults;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* 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 io.tabular.iceberg.connect.api;

public interface CommittableSupplier {
Committable committable();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* 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 io.tabular.iceberg.connect.api;

public interface Committer {
void commit(CommittableSupplier committableSupplier);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* 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 io.tabular.iceberg.connect.api;

import java.util.Collection;
import org.apache.kafka.connect.sink.SinkRecord;

public interface Writer extends CommittableSupplier {
void write(Collection<SinkRecord> sinkRecords);
}
Loading

0 comments on commit 375bc15

Please sign in to comment.