Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Refactor into Writer and Committer #234

Merged
merged 12 commits into from
Apr 25, 2024
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 @@ -18,27 +18,14 @@
*/
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.channel.Task;
import io.tabular.iceberg.connect.channel.TaskImpl;
import io.tabular.iceberg.connect.data.Utilities;
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 +36,7 @@ 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 Task task;

@Override
public String version() {
Expand All @@ -75,114 +48,39 @@ 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);
fqtab marked this conversation as resolved.
Show resolved Hide resolved
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) {
fqtab marked this conversation as resolved.
Show resolved Hide resolved
// 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"));

return partitions.contains(firstTopicPartition);
private void clearState() {
Utilities.close(task);
task = 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.task = new TaskImpl(context, config);
}

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

@Override
public void flush(Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
processControlEvents();
}
Comment on lines -162 to -164
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed the flush method entirely.
All the work now happens in put.


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();
Copy link
Contributor Author

@fqtab fqtab Apr 9, 2024

Choose a reason for hiding this comment

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

Not committing to the connect consumer group via this method anymore, all consumer-offset-commits are managed manually.

}

@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
Expand Up @@ -18,18 +18,17 @@
*/
package io.tabular.iceberg.connect.channel;

import static java.util.stream.Collectors.toList;

import io.tabular.iceberg.connect.IcebergSinkConfig;
import io.tabular.iceberg.connect.data.Offset;
import io.tabular.iceberg.connect.events.Event;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.Pair;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
Expand All @@ -46,7 +45,6 @@ public abstract class Channel {
private static final Logger LOG = LoggerFactory.getLogger(Channel.class);

private final String controlTopic;
private final String controlGroupId;
private final String groupId;
private final Producer<String, byte[]> producer;
private final Consumer<String, byte[]> consumer;
Expand All @@ -60,26 +58,27 @@ public Channel(
IcebergSinkConfig config,
KafkaClientFactory clientFactory) {
this.controlTopic = config.controlTopic();
this.controlGroupId = config.controlGroupId();
this.groupId = config.controlGroupId();

String transactionalId = name + config.transactionalSuffix();
this.producer = clientFactory.createProducer(transactionalId);
Pair<UUID, Producer<String, byte[]>> pair = clientFactory.createProducer(transactionalId);
this.producer = pair.second();
this.consumer = clientFactory.createConsumer(consumerGroupId);
consumer.subscribe(ImmutableList.of(controlTopic));
this.admin = clientFactory.createAdmin();

this.producerId = UUID.randomUUID().toString();
this.producerId = pair.first().toString();
}

protected void send(Event event) {
send(ImmutableList.of(event), ImmutableMap.of());
send(ImmutableList.of(event), ImmutableMap.of(), null);
}

protected void send(List<Event> events, Map<TopicPartition, Offset> sourceOffsets) {
Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = Maps.newHashMap();
sourceOffsets.forEach((k, v) -> offsetsToCommit.put(k, new OffsetAndMetadata(v.offset())));

List<ProducerRecord<String, byte[]>> recordList =
protected void send(
List<Event> events,
Map<TopicPartition, OffsetAndMetadata> consumerOffsets,
ConsumerGroupMetadata consumerGroupMetadata) {
sendAndCommitOffsets(
events.stream()
.map(
event -> {
Expand All @@ -88,16 +87,22 @@ protected void send(List<Event> events, Map<TopicPartition, Offset> sourceOffset
// key by producer ID to keep event order
return new ProducerRecord<>(controlTopic, producerId, data);
})
.collect(toList());
.collect(Collectors.toList()),
consumerOffsets,
consumerGroupMetadata);
}

private void sendAndCommitOffsets(
List<ProducerRecord<String, byte[]>> producerRecords,
Map<TopicPartition, OffsetAndMetadata> consumerOffsets,
ConsumerGroupMetadata consumerGroupMetadata) {
synchronized (producer) {
producer.beginTransaction();
try {
recordList.forEach(producer::send);
if (!sourceOffsets.isEmpty()) {
// TODO: this doesn't fence zombies
producer.sendOffsetsToTransaction(
offsetsToCommit, new ConsumerGroupMetadata(controlGroupId));
producerRecords.forEach(producer::send);
producer.flush();
if (!consumerOffsets.isEmpty()) {
producer.sendOffsetsToTransaction(consumerOffsets, consumerGroupMetadata);
}
producer.commitTransaction();
} catch (Exception e) {
Expand All @@ -111,9 +116,8 @@ protected void send(List<Event> events, Map<TopicPartition, Offset> sourceOffset
}
}

protected abstract boolean receive(Envelope envelope);

protected void consumeAvailable(Duration pollDuration) {
protected void consumeAvailable(
Duration pollDuration, java.util.function.Function<Envelope, Boolean> receiveFn) {
ConsumerRecords<String, byte[]> records = consumer.poll(pollDuration);
while (!records.isEmpty()) {
records.forEach(
Expand All @@ -126,7 +130,7 @@ record -> {

if (event.groupId().equals(groupId)) {
LOG.debug("Received event of type: {}", event.type().name());
if (receive(new Envelope(event, record.partition(), record.offset()))) {
if (receiveFn.apply(new Envelope(event, record.partition(), record.offset()))) {
LOG.info("Handled event of type: {}", event.type().name());
}
}
Expand All @@ -152,13 +156,6 @@ protected Admin admin() {
return admin;
}

public void start() {
consumer.subscribe(ImmutableList.of(controlTopic));

// initial poll with longer duration so the consumer will initialize...
consumeAvailable(Duration.ofMillis(1000));
fqtab marked this conversation as resolved.
Show resolved Hide resolved
}

public void stop() {
LOG.info("Channel stopping");
producer.close();
Expand Down
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.channel;

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;

class Committable {

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

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;
}
}
Loading
Loading