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

Handling NO Coordinator Scenario and Data Loss in the current Design #11298

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
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 @@ -19,14 +19,17 @@
package org.apache.iceberg.connect;

import java.util.Collection;
import org.apache.iceberg.catalog.Catalog;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTaskContext;
import org.apache.kafka.common.TopicPartition;

public interface Committer {
void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context);
void start(ResourceType resourceType);

void stop();
void stop(ResourceType resourceType);

void save(Collection<SinkRecord> sinkRecords);

boolean isLeader(Collection<TopicPartition> currentAssignedPartitions);

void syncLastCommittedOffsets();
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,13 @@
*/
package org.apache.iceberg.connect;

import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.connect.channel.CommitterImpl;
import org.apache.kafka.connect.sink.SinkTaskContext;

class CommitterFactory {
static Committer createCommitter(IcebergSinkConfig config) {
return new CommitterImpl();
static Committer createCommitter(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) {
return new CommitterImpl(catalog, config, context);
}

private CommitterFactory() {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.util.Collection;
import java.util.Map;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
Expand All @@ -46,26 +45,31 @@ public String version() {
@Override
public void start(Map<String, String> props) {
this.config = new IcebergSinkConfig(props);
catalog = CatalogUtils.loadCatalog(config);
committer = CommitterFactory.createCommitter(catalog, config, context);
}

@Override
public void open(Collection<TopicPartition> partitions) {
Preconditions.checkArgument(catalog == null, "Catalog already open");
Preconditions.checkArgument(committer == null, "Committer already open");

catalog = CatalogUtils.loadCatalog(config);
committer = CommitterFactory.createCommitter(config);
committer.start(catalog, config, context);
if(committer.isLeader(partitions)) {
committer.start(ResourceType.COORDINATOR);
}
committer.syncLastCommittedOffsets();
}

@Override
public void close(Collection<TopicPartition> partitions) {
close();
// We need to close worker here in every case to ensure exactly once.
committer.stop(ResourceType.WORKER);
if(committer.isLeader(partitions)) {
committer.stop(ResourceType.COORDINATOR);
}
}

private void close() {
if (committer != null) {
committer.stop();
committer.stop(ResourceType.WORKER);
committer.stop(ResourceType.COORDINATOR);
committer = null;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
*
* * 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 org.apache.iceberg.connect;

public enum ResourceType {
WORKER,
COORDINATOR
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,19 @@

import java.util.Collection;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.connect.Committer;
import org.apache.iceberg.connect.IcebergSinkConfig;
import org.apache.iceberg.connect.ResourceType;
import org.apache.iceberg.connect.data.SinkWriter;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.ConsumerGroupDescription;
import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions;
import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult;
import org.apache.kafka.clients.admin.MemberDescription;
import org.apache.kafka.common.ConsumerGroupState;
import org.apache.kafka.common.TopicPartition;
Expand All @@ -37,12 +42,33 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

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

public class CommitterImpl implements Committer {

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

private CoordinatorThread coordinatorThread;
private Worker worker;
private final Catalog catalog;
private final IcebergSinkConfig config;
private final SinkTaskContext context;
private final KafkaClientFactory clientFactory;
private Collection<MemberDescription> membersWhenWorkerIsCoordinator;

public CommitterImpl(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) {
this.catalog = catalog;
this.config = config;
this.context = context;
this.clientFactory = new KafkaClientFactory(config.kafkaProps());
}

public CommitterImpl() {
this.catalog = null;
this.config = null;
this.context = null;
this.clientFactory = null;
}

static class TopicPartitionComparator implements Comparator<TopicPartition> {

Expand All @@ -57,49 +83,67 @@ public int compare(TopicPartition o1, TopicPartition o2) {
}

@Override
public void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) {
KafkaClientFactory clientFactory = new KafkaClientFactory(config.kafkaProps());

public boolean isLeader(Collection<TopicPartition> currentAssignedPartitions) {
ConsumerGroupDescription groupDesc;
try (Admin admin = clientFactory.createAdmin()) {
groupDesc = KafkaUtils.consumerGroupDescription(config.connectGroupId(), admin);
}

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

LOG.info("Starting commit worker");
SinkWriter sinkWriter = new SinkWriter(catalog, config);
worker = new Worker(config, clientFactory, sinkWriter, context);
worker.start();
@Override
public void start(ResourceType resourceType) {
switch (resourceType) {
case WORKER:
startWorker();
break;
case COORDINATOR:
startCoordinator();
}
}

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

@Override
public void stop() {
if (worker != null) {
worker.stop();
worker = null;
public void syncLastCommittedOffsets() {
Map<TopicPartition, Long> stableConsumerOffsets;
try (Admin admin = clientFactory.createAdmin()) {
ListConsumerGroupOffsetsResult response =
admin.listConsumerGroupOffsets(
config.connectGroupId(), new ListConsumerGroupOffsetsOptions().requireStable(true));
stableConsumerOffsets =
response.partitionsToOffsetAndMetadata().get().entrySet().stream()
.filter(entry -> context.assignment().contains(entry.getKey()))
.collect(toMap(Map.Entry::getKey, entry -> entry.getValue().offset()));
} catch (InterruptedException | ExecutionException e) {
throw new ConnectException(e);
}
context.offset(stableConsumerOffsets);
}

if (coordinatorThread != null) {
coordinatorThread.terminate();
coordinatorThread = null;
@Override
public void stop(ResourceType resourceType) {
switch (resourceType) {
case WORKER:
stopWorker();
break;
case COORDINATOR:
stopCoordinator();
}
}

Expand All @@ -108,11 +152,11 @@ boolean isLeader(Collection<MemberDescription> members, Collection<TopicPartitio
// 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"));
members.stream()
.flatMap(member -> member.assignment().topicPartitions().stream())
.min(new TopicPartitionComparator())
.orElseThrow(
() -> new ConnectException("No partitions assigned, cannot determine leader"));

return partitions.contains(firstTopicPartition);
}
Expand All @@ -125,4 +169,34 @@ private void processControlEvents() {
worker.process();
}
}
}

private void startWorker() {
if(null == this.worker) {
LOG.info("Starting commit worker");
SinkWriter sinkWriter = new SinkWriter(catalog, config);
worker = new Worker(config, clientFactory, sinkWriter, context);
worker.start();
}
}

private void startCoordinator() {
LOG.info("Task elected leader, starting commit coordinator");
Coordinator coordinator = new Coordinator(catalog, config, membersWhenWorkerIsCoordinator, clientFactory, context);
coordinatorThread = new CoordinatorThread(coordinator);
coordinatorThread.start();
}

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

private void stopCoordinator() {
if (coordinatorThread != null) {
coordinatorThread.terminate();
coordinatorThread = null;
}
}
}