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 @@ -68,6 +68,7 @@
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.python.env.PythonDependencyInfo;
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StateSnapshotContext;
import org.apache.flink.runtime.state.VoidNamespace;
Expand Down Expand Up @@ -916,6 +917,9 @@ private void tryResumeProcessActionTasks() throws Exception {
Iterable<Object> keys = currentProcessingKeysOpState.get();
if (keys != null) {
for (Object key : keys) {
if (!isKeyOwnedByCurrentSubtask(key)) {
continue;
}
keySegmentQueue.addKeyToLastSegment(key);
mailboxExecutor.submit(
() -> tryProcessActionTaskForKey(key), "process action task");
Expand Down Expand Up @@ -1122,6 +1126,18 @@ private void maybeInitActionStateStore() {
}
}

private boolean isKeyOwnedByCurrentSubtask(Object key) {
int maxParallelism = getRuntimeContext().getTaskInfo().getMaxNumberOfParallelSubtasks();
int parallelism = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks();
int subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();

int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism);
int owner =
KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(
maxParallelism, parallelism, keyGroup);
return owner == subtaskIndex;
}

/** Failed to execute Action task. */
public static class ActionTaskExecutionException extends Exception {
public ActionTaskExecutionException(String message, Throwable cause) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,9 +33,12 @@
import org.apache.flink.agents.runtime.eventlog.FileEventLogger;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox;
import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
import org.apache.flink.util.ExceptionUtils;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -151,6 +154,71 @@ void testDifferentKeyDataCanRunConcurrently() throws Exception {
}
}

@Test
void testRestoreOnlyResumesKeysOwnedByCurrentSubtask() throws Exception {
final int maxParallelism = 4;
final int oldParallelism = 1;
final int newParallelism = 2;
final long key = 1L;

OperatorSubtaskState snapshot;
try (KeyedOneInputStreamOperatorTestHarness<Long, Long, Object> testHarness =
new KeyedOneInputStreamOperatorTestHarness<>(
new ActionExecutionOperatorFactory(TestAgent.getAgentPlan(false), true),
(KeySelector<Long, Long>) value -> value,
TypeInformation.of(Long.class),
maxParallelism,
oldParallelism,
0)) {
testHarness.open();
testHarness.processElement(new StreamRecord<>(key));
assertThat(testHarness.getTaskMailbox().size()).isEqualTo(1);
snapshot = testHarness.snapshot(1L, 1L);
}

int ownerSubtask =
KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(
maxParallelism,
newParallelism,
KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism));
int nonOwnerSubtask = 1 - ownerSubtask;

OperatorSubtaskState ownerState =
AbstractStreamOperatorTestHarness.repartitionOperatorState(
snapshot, maxParallelism, oldParallelism, newParallelism, ownerSubtask);
OperatorSubtaskState nonOwnerState =
AbstractStreamOperatorTestHarness.repartitionOperatorState(
snapshot, maxParallelism, oldParallelism, newParallelism, nonOwnerSubtask);

try (KeyedOneInputStreamOperatorTestHarness<Long, Long, Object> ownerHarness =
new KeyedOneInputStreamOperatorTestHarness<>(
new ActionExecutionOperatorFactory(
TestAgent.getAgentPlan(false), true),
(KeySelector<Long, Long>) value -> value,
TypeInformation.of(Long.class),
maxParallelism,
newParallelism,
ownerSubtask);
KeyedOneInputStreamOperatorTestHarness<Long, Long, Object> nonOwnerHarness =
new KeyedOneInputStreamOperatorTestHarness<>(
new ActionExecutionOperatorFactory(
TestAgent.getAgentPlan(false), true),
(KeySelector<Long, Long>) value -> value,
TypeInformation.of(Long.class),
maxParallelism,
newParallelism,
nonOwnerSubtask)) {
ownerHarness.initializeState(ownerState);
nonOwnerHarness.initializeState(nonOwnerState);

ownerHarness.open();
nonOwnerHarness.open();

assertThat(ownerHarness.getTaskMailbox().size()).isEqualTo(1);
assertThat(nonOwnerHarness.getTaskMailbox().size()).isZero();
}
}

@Test
void testMemoryAccessProhibitedOutsideMailboxThread() throws Exception {
try (KeyedOneInputStreamOperatorTestHarness<Long, Long, Object> testHarness =
Expand Down
Loading