Skip to content
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 @@ -21,20 +21,44 @@
import org.apache.flink.annotation.Public;

import java.io.Serializable;
import java.util.Collections;
import java.util.List;
import java.util.Objects;

/** A container class hosting the information of a {@link SourceReader}. */
/**
* A container class hosting the information of a {@link SourceReader}.
*
* <p>The {@code reportedSplitsOnRegistration} can only be provided when the source implements
* {@link SupportSplitReassignmentOnRecovery}.
*/
@Public
public final class ReaderInfo implements Serializable {

private static final long serialVersionUID = 1L;

private final int subtaskId;
private final String location;
private final List<SourceSplit> reportedSplitsOnRegistration;

public ReaderInfo(int subtaskId, String location) {
this(subtaskId, location, Collections.emptyList());
}

ReaderInfo(int subtaskId, String location, List<SourceSplit> splits) {
this.subtaskId = subtaskId;
this.location = location;
this.reportedSplitsOnRegistration = splits;
}

@SuppressWarnings("unchecked")
public static <SplitT extends SourceSplit> ReaderInfo createReaderInfo(
int subtaskId, String location, List<SplitT> splits) {
return new ReaderInfo(subtaskId, location, (List<SourceSplit>) splits);
}

@SuppressWarnings("unchecked")
public <SplitT extends SourceSplit> List<SplitT> getReportedSplitsOnRegistration() {
return (List<SplitT>) reportedSplitsOnRegistration;
}

/**
Expand All @@ -52,16 +76,18 @@ public String getLocation() {
}

@Override
public int hashCode() {
return Objects.hash(subtaskId, location);
public boolean equals(Object o) {
if (!(o instanceof ReaderInfo)) {
return false;
}
ReaderInfo that = (ReaderInfo) o;
return subtaskId == that.subtaskId
&& Objects.equals(location, that.location)
&& Objects.equals(reportedSplitsOnRegistration, that.reportedSplitsOnRegistration);
}

@Override
public boolean equals(Object obj) {
if (!(obj instanceof ReaderInfo)) {
return false;
}
ReaderInfo other = (ReaderInfo) obj;
return subtaskId == other.subtaskId && location.equals(other.location);
public int hashCode() {
return Objects.hash(subtaskId, location, reportedSplitsOnRegistration);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* 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.flink.api.connector.source;

import org.apache.flink.annotation.PublicEvolving;

/**
* A decorative interface {@link Source}. Implementing this interface indicates that the source
* operator needs to report splits to the enumerator and receive reassignment.
*/
@PublicEvolving
public interface SupportSplitReassignmentOnRecovery {}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.api.connector.source.mocks;

import org.apache.flink.api.connector.source.ReaderInfo;
import org.apache.flink.api.connector.source.SourceEvent;
import org.apache.flink.api.connector.source.SplitEnumerator;
import org.apache.flink.api.connector.source.SplitEnumeratorContext;
Expand All @@ -28,20 +29,21 @@

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.stream.Collectors;

/** A mock {@link SplitEnumerator} for unit tests. */
public class MockSplitEnumerator
implements SplitEnumerator<MockSourceSplit, Set<MockSourceSplit>>, SupportsBatchSnapshot {
private final SortedSet<MockSourceSplit> unassignedSplits;
// 扩成16个partition, unas
private final Map<Integer, Set<MockSourceSplit>> pendingSplitAssignment;
private final Map<String, Integer> globalSplitAssignment;
private final SplitEnumeratorContext<MockSourceSplit> enumContext;
private final List<SourceEvent> handledSourceEvent;
private final List<Long> successfulCheckpoints;
Expand All @@ -50,22 +52,24 @@ public class MockSplitEnumerator

public MockSplitEnumerator(int numSplits, SplitEnumeratorContext<MockSourceSplit> enumContext) {
this(new HashSet<>(), enumContext);
List<MockSourceSplit> unassignedSplits = new ArrayList<>();
for (int i = 0; i < numSplits; i++) {
unassignedSplits.add(new MockSourceSplit(i));
}
calculateAndPutPendingAssignments(unassignedSplits);
}

public MockSplitEnumerator(
Set<MockSourceSplit> unassignedSplits,
SplitEnumeratorContext<MockSourceSplit> enumContext) {
this.unassignedSplits =
new TreeSet<>(Comparator.comparingInt(o -> Integer.parseInt(o.splitId())));
this.unassignedSplits.addAll(unassignedSplits);
this.pendingSplitAssignment = new HashMap<>();
this.globalSplitAssignment = new HashMap<>();
this.enumContext = enumContext;
this.handledSourceEvent = new ArrayList<>();
this.successfulCheckpoints = new ArrayList<>();
this.started = false;
this.closed = false;
calculateAndPutPendingAssignments(unassignedSplits);
}

@Override
Expand All @@ -83,25 +87,36 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {

@Override
public void addSplitsBack(List<MockSourceSplit> splits, int subtaskId) {
unassignedSplits.addAll(splits);
// add back to same subtaskId.
putPendingAssignments(subtaskId, splits);
}

@Override
public void addReader(int subtaskId) {
List<MockSourceSplit> assignment = new ArrayList<>();
for (MockSourceSplit split : unassignedSplits) {
if (Integer.parseInt(split.splitId()) % enumContext.currentParallelism() == subtaskId) {
assignment.add(split);
ReaderInfo readerInfo = enumContext.registeredReaders().get(subtaskId);
List<MockSourceSplit> splitsOnRecovery = readerInfo.getReportedSplitsOnRegistration();

List<MockSourceSplit> redistributedSplits = new ArrayList<>();
List<MockSourceSplit> addBackSplits = new ArrayList<>();
for (MockSourceSplit split : splitsOnRecovery) {
if (!globalSplitAssignment.containsKey(split.splitId())) {
// if split not existed in globalSplitAssignment, mean that it's registered first
// time, can be redistibuted.
redistributedSplits.add(split);
} else if (!globalSplitAssignment.containsKey(split.splitId())) {
// if split already is assigned to other substaskId, just ignore it. Otherwise,
// addback to this subtaskId again.
addBackSplits.add(split);
}
}
enumContext.assignSplits(
new SplitsAssignment<>(Collections.singletonMap(subtaskId, assignment)));
unassignedSplits.removeAll(assignment);
calculateAndPutPendingAssignments(redistributedSplits);
putPendingAssignments(subtaskId, addBackSplits);
assignAllSplits();
}

@Override
public Set<MockSourceSplit> snapshotState(long checkpointId) {
return unassignedSplits;
return getUnassignedSplits();
}

@Override
Expand All @@ -114,11 +129,6 @@ public void close() throws IOException {
this.closed = true;
}

public void addNewSplits(List<MockSourceSplit> newSplits) {
unassignedSplits.addAll(newSplits);
assignAllSplits();
}

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

public boolean started() {
Expand All @@ -130,7 +140,9 @@ public boolean closed() {
}

public Set<MockSourceSplit> getUnassignedSplits() {
return unassignedSplits;
return pendingSplitAssignment.values().stream()
.flatMap(Set::stream)
.collect(Collectors.toSet());
}

public List<SourceEvent> getHandledSourceEvent() {
Expand All @@ -145,17 +157,27 @@ public List<Long> getSuccessfulCheckpoints() {

private void assignAllSplits() {
Map<Integer, List<MockSourceSplit>> assignment = new HashMap<>();
unassignedSplits.forEach(
split -> {
int subtaskId =
Integer.parseInt(split.splitId()) % enumContext.currentParallelism();
if (enumContext.registeredReaders().containsKey(subtaskId)) {
assignment
.computeIfAbsent(subtaskId, ignored -> new ArrayList<>())
.add(split);
}
});
for (Map.Entry<Integer, Set<MockSourceSplit>> iter : pendingSplitAssignment.entrySet()) {
Integer subtaskId = iter.getKey();
if (enumContext.registeredReaders().containsKey(subtaskId)) {
assignment.put(subtaskId, new ArrayList<>(iter.getValue()));
}
}
enumContext.assignSplits(new SplitsAssignment<>(assignment));
assignment.values().forEach(l -> unassignedSplits.removeAll(l));
assignment.keySet().forEach(pendingSplitAssignment::remove);
}

private void calculateAndPutPendingAssignments(Collection<MockSourceSplit> newSplits) {
for (MockSourceSplit split : newSplits) {
int subtaskId = Integer.parseInt(split.splitId()) % enumContext.currentParallelism();
putPendingAssignments(subtaskId, Collections.singletonList(split));
}
}

private void putPendingAssignments(int subtaskId, Collection<MockSourceSplit> splits) {
Set<MockSourceSplit> pendingSplits =
pendingSplitAssignment.computeIfAbsent(subtaskId, HashSet::new);
pendingSplits.addAll(splits);
splits.forEach(split -> globalSplitAssignment.put(split.splitId(), subtaskId));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,9 @@ public class SourceCoordinator<SplitT extends SourceSplit, EnumChkT>
/** The Source that is associated with this SourceCoordinator. */
private final Source<?, SplitT, EnumChkT> source;

/** The serializer that handles the serde of the split. */
private final SimpleVersionedSerializer<SplitT> splitSerializer;

/** The serializer that handles the serde of the SplitEnumerator checkpoints. */
private final SimpleVersionedSerializer<EnumChkT> enumCheckpointSerializer;

Expand Down Expand Up @@ -163,6 +166,7 @@ public SourceCoordinator(
this.operatorName = operatorName;
this.source = source;
this.enumCheckpointSerializer = source.getEnumeratorCheckpointSerializer();
this.splitSerializer = source.getSplitSerializer();
this.context = context;
this.coordinatorStore = coordinatorStore;
this.watermarkAlignmentParams = watermarkAlignmentParams;
Expand Down Expand Up @@ -427,7 +431,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r
// assignments
byte[] assignmentData =
context.getAssignmentTracker()
.snapshotState(source.getSplitSerializer());
.snapshotState(splitSerializer);
out.writeInt(assignmentData.length);
out.write(assignmentData);

Expand Down Expand Up @@ -680,7 +684,7 @@ private void handleSourceEvent(int subtask, int attemptNumber, SourceEvent event
}

private void handleReaderRegistrationEvent(
int subtask, int attemptNumber, ReaderRegistrationEvent event) {
int subtask, int attemptNumber, ReaderRegistrationEvent event) throws Exception {
checkArgument(subtask == event.subtaskId());

LOG.info(
Expand All @@ -692,7 +696,8 @@ private void handleReaderRegistrationEvent(

final boolean subtaskReaderExisted =
context.registeredReadersOfAttempts().containsKey(subtask);
context.registerSourceReader(subtask, attemptNumber, event.location());
context.registerSourceReader(
subtask, attemptNumber, event.location(), event.splits(splitSerializer));
if (!subtaskReaderExisted) {
enumerator.addReader(event.subtaskId());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -465,16 +465,18 @@ void onCheckpoint(long checkpointId) throws Exception {
* @param subtaskId the subtask id of the source reader.
* @param attemptNumber the attempt number of the source reader.
* @param location the location of the source reader.
* @param splits the split restored from source reader's state.
*/
void registerSourceReader(int subtaskId, int attemptNumber, String location) {
void registerSourceReader(
int subtaskId, int attemptNumber, String location, List<SplitT> splits) {
final Map<Integer, ReaderInfo> attemptReaders =
registeredReaders.computeIfAbsent(subtaskId, k -> new ConcurrentHashMap<>());
checkState(
!attemptReaders.containsKey(attemptNumber),
"ReaderInfo of subtask %s (#%s) already exists.",
subtaskId,
attemptNumber);
attemptReaders.put(attemptNumber, new ReaderInfo(subtaskId, location));
attemptReaders.put(attemptNumber, ReaderInfo.createReaderInfo(subtaskId, location, splits));

sendCachedSplitsToNewlyRegisteredReader(subtaskId, attemptNumber);
}
Expand Down
Loading