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

[FLINK-20194][connector/kafka] Fix the KafkaSourceReader offset commit. #14142

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -172,11 +171,12 @@ private void finishCurrentFetch(final RecordsWithSplitIds<E> fetch, final Reader
final Set<String> finishedSplits = fetch.finishedSplits();
if (!finishedSplits.isEmpty()) {
LOG.info("Finished reading split(s) {}", finishedSplits);
Map<String, SplitStateT> stateOfFinishedSplits = new HashMap<>();
for (String finishedSplitId : finishedSplits) {
splitStates.remove(finishedSplitId);
stateOfFinishedSplits.put(finishedSplitId, splitStates.remove(finishedSplitId).state);
output.releaseOutputForSplit(finishedSplitId);
}
onSplitFinished(finishedSplits);
onSplitFinished(stateOfFinishedSplits);
}

fetch.recycle();
Expand Down Expand Up @@ -251,7 +251,7 @@ public int getNumberOfCurrentlyAssignedSplits() {
/**
* Handles the finished splits to clean the state if needed.
*/
protected abstract void onSplitFinished(Collection<String> finishedSplitIds);
protected abstract void onSplitFinished(Map<String, SplitStateT> finishedSplitIds);

/**
* When new splits are added to the reader. The initialize the state of the new splits.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ public SingleThreadFetcherManager(

@Override
public void addSplits(List<SplitT> splitsToAdd) {
SplitFetcher<E, SplitT> fetcher = fetchers.get(0);
SplitFetcher<E, SplitT> fetcher = getRunningFetcher();
if (fetcher == null) {
fetcher = createSplitFetcher();
// Add the splits to the fetchers.
Expand All @@ -64,4 +64,8 @@ public void addSplits(List<SplitT> splitsToAdd) {
fetcher.addSplits(splitsToAdd);
}
}

protected SplitFetcher<E, SplitT> getRunningFetcher() {
return fetchers.isEmpty() ? null : fetchers.values().iterator().next();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.connector.base.source.reader.fetcher;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.connector.source.SourceSplit;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.SourceReaderBase;
Expand Down Expand Up @@ -185,4 +186,11 @@ public void checkErrors() {
uncaughtFetcherException.get());
}
}

// -----------------------

@VisibleForTesting
public int getNumAliveFetchers() {
return fetchers.size();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,9 @@

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.function.Supplier;

Expand Down Expand Up @@ -296,7 +296,7 @@ private Configuration getConfig() {
public void notifyCheckpointComplete(long checkpointId) {}

@Override
protected void onSplitFinished(Collection<String> finishedSplitIds) {}
protected void onSplitFinished(Map<String, TestingSourceSplit> finishedSplitIds) {}

@Override
protected TestingSourceSplit initializedState(TestingSourceSplit split) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;

import java.util.Collection;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;

Expand All @@ -52,7 +52,7 @@ public MockSourceReader(FutureCompletingBlockingQueue<RecordsWithSplitIds<int[]>
}

@Override
protected void onSplitFinished(Collection<String> finishedSplitIds) {
protected void onSplitFinished(Map<String, AtomicInteger> finishedSplitIds) {

}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import org.apache.flink.connector.file.src.reader.BulkFormat;
import org.apache.flink.connector.file.src.util.RecordAndPosition;

import java.util.Collection;
import java.util.Map;

/**
* A {@link SourceReader} that read records from {@link FileSourceSplit}.
Expand All @@ -54,7 +54,7 @@ public void start() {
}

@Override
protected void onSplitFinished(Collection<String> finishedSplitIds) {
protected void onSplitFinished(Map<String, FileSourceSplitState<SplitT>> finishedSplitIds) {
context.sendSplitRequest();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,12 +35,14 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.function.Supplier;

/**
Expand All @@ -49,7 +51,10 @@
public class KafkaSourceReader<T>
extends SingleThreadMultiplexSourceReaderBase<Tuple3<T, Long, Long>, T, KafkaPartitionSplit, KafkaPartitionSplitState> {
private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceReader.class);
// These maps need to be concurrent because it will be accessed by both the main thread
// and the split fetcher thread in the callback.
private final SortedMap<Long, Map<TopicPartition, OffsetAndMetadata>> offsetsToCommit;
private final ConcurrentMap<TopicPartition, OffsetAndMetadata> offsetsOfFinishedSplits;

public KafkaSourceReader(
FutureCompletingBlockingQueue<RecordsWithSplitIds<Tuple3<T, Long, Long>>> elementsQueue,
Expand All @@ -63,37 +68,55 @@ public KafkaSourceReader(
recordEmitter,
config,
context);
this.offsetsToCommit = new TreeMap<>();
this.offsetsToCommit = Collections.synchronizedSortedMap(new TreeMap<>());
this.offsetsOfFinishedSplits = new ConcurrentHashMap<>();
}

@Override
protected void onSplitFinished(Collection<String> finishedSplitIds) {

protected void onSplitFinished(Map<String, KafkaPartitionSplitState> finishedSplitIds) {
finishedSplitIds.forEach((ignored, splitState) -> {
offsetsOfFinishedSplits.put(
splitState.getTopicPartition(),
new OffsetAndMetadata(splitState.getCurrentOffset()));
});
}

@Override
public List<KafkaPartitionSplit> snapshotState(long checkpointId) {
List<KafkaPartitionSplit> splits = super.snapshotState(checkpointId);
for (KafkaPartitionSplit split : splits) {
offsetsToCommit
.compute(checkpointId, (ignoredKey, ignoredValue) -> new HashMap<>())
.put(
split.getTopicPartition(),
new OffsetAndMetadata(split.getStartingOffset(), null));
if (splits.isEmpty() && offsetsOfFinishedSplits.isEmpty()) {
offsetsToCommit.put(checkpointId, Collections.emptyMap());
} else {
Map<TopicPartition, OffsetAndMetadata> offsetsMap =
offsetsToCommit.computeIfAbsent(checkpointId, id -> new HashMap<>());
// Put the offsets of the active splits.
for (KafkaPartitionSplit split : splits) {
offsetsMap.put(
split.getTopicPartition(),
new OffsetAndMetadata(split.getStartingOffset(), null));
}
// Put offsets of all the finished splits.
offsetsMap.putAll(offsetsOfFinishedSplits);
}
return splits;
}

@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
LOG.info("Committing offsets for checkpoint {}", checkpointId);
((KafkaSourceFetcherManager<T>) splitFetcherManager).commitOffsets(
offsetsToCommit.get(checkpointId),
(ignored, e) -> {
if (e != null) {
LOG.warn(
"Failed to commit consumer offsets for checkpoint {}",
checkpointId);
LOG.warn("Failed to commit consumer offsets for checkpoint {}", checkpointId, e);
} else {
LOG.debug("Successfully committed offsets for checkpoint {}", checkpointId);
// If the finished topic partition has been committed, we remove it
// from the offsets of finsihed splits map.
Map<TopicPartition, OffsetAndMetadata> committedPartitions =
offsetsToCommit.get(checkpointId);
offsetsOfFinishedSplits.entrySet().removeIf(
entry -> committedPartitions.containsKey(entry.getKey()));
while (!offsetsToCommit.isEmpty() && offsetsToCommit.firstKey() <= checkpointId) {
offsetsToCommit.remove(offsetsToCommit.firstKey());
}
Expand All @@ -117,4 +140,9 @@ protected KafkaPartitionSplit toSplitType(String splitId, KafkaPartitionSplitSta
SortedMap<Long, Map<TopicPartition, OffsetAndMetadata>> getOffsetsToCommit() {
return offsetsToCommit;
}

@VisibleForTesting
int getNumAliveFetchers() {
return splitFetcherManager.getNumAliveFetchers();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.Map;
Expand All @@ -44,6 +46,7 @@
*/
public class KafkaSourceFetcherManager<T>
extends SingleThreadFetcherManager<Tuple3<T, Long, Long>, KafkaPartitionSplit> {
private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceFetcherManager.class);

/**
* Creates a new SplitFetcherManager with a single I/O threads.
Expand All @@ -62,7 +65,25 @@ public KafkaSourceFetcherManager(
public void commitOffsets(
Map<TopicPartition, OffsetAndMetadata> offsetsToCommit,
OffsetCommitCallback callback) {
LOG.debug("Committing offsets {}", offsetsToCommit);
if (offsetsToCommit.isEmpty()) {
return;
}
SplitFetcher<Tuple3<T, Long, Long>, KafkaPartitionSplit> splitFetcher = fetchers.get(0);
if (splitFetcher != null) {
// The fetcher thread is still running. This should be the majority of the cases.
enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback);
} else {
splitFetcher = createSplitFetcher();
enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback);
startFetcher(splitFetcher);
}
}

private void enqueueOffsetsCommitTask(
SplitFetcher<Tuple3<T, Long, Long>, KafkaPartitionSplit> splitFetcher,
Map<TopicPartition, OffsetAndMetadata> offsetsToCommit,
OffsetCommitCallback callback) {
KafkaPartitionSplitReader<T> kafkaReader =
(KafkaPartitionSplitReader<T>) splitFetcher.getSplitReader();

Expand All @@ -74,7 +95,8 @@ public boolean run() throws IOException {
}

@Override
public void wakeUp() {}
public void wakeUp() {
}
});
}
}