Skip to content

Commit

Permalink
[FLINK-36963][Runtime] Fix wrong context maintain around `asyncProces…
Browse files Browse the repository at this point in the history
…sWithKey`
  • Loading branch information
Zakelly committed Dec 26, 2024
1 parent 56ba88b commit b582489
Show file tree
Hide file tree
Showing 5 changed files with 22 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -503,6 +503,6 @@ public void close() throws IOException {

/** A listener listens the key context switch. */
public interface SwitchContextListener<K> {
void switchContext(RecordContext<K> context);
void switchContext(@Nullable RecordContext<K> context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ public abstract class AbstractAsyncStateStreamOperator<OUT> extends AbstractStre

private AsyncExecutionController asyncExecutionController;

/** Act as a cache for {@link #setAsyncKeyedContextElement} and {@link #postProcessElement}. */
private RecordContext currentProcessingContext;

private Environment environment;
Expand Down Expand Up @@ -175,20 +176,18 @@ public final void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> proc
@Override
@SuppressWarnings("unchecked")
public <K> void asyncProcessWithKey(K key, ThrowingRunnable<Exception> processing) {
RecordContext<K> previousContext = currentProcessingContext;

RecordContext<K> oldContext = asyncExecutionController.getCurrentContext();
// build a context and switch to the new context
currentProcessingContext = asyncExecutionController.buildContext(null, key, true);
currentProcessingContext.retain();
asyncExecutionController.setCurrentContext(currentProcessingContext);
RecordContext<K> newContext = asyncExecutionController.buildContext(null, key, true);
newContext.retain();
asyncExecutionController.setCurrentContext(newContext);
// Same logic as RECORD_ORDER, since FIRST_STATE_ORDER is problematic when the call's key
// pass the same key in.
preserveRecordOrderAndProcess(processing);
postProcessElement();
newContext.release();

// switch to original context
asyncExecutionController.setCurrentContext(previousContext);
currentProcessingContext = previousContext;
asyncExecutionController.setCurrentContext(oldContext);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ public abstract class AbstractAsyncStateStreamOperatorV2<OUT> extends AbstractSt
private final Environment environment;
private AsyncExecutionController asyncExecutionController;

/** Act as a cache for {@link #setAsyncKeyedContextElement} and {@link #postProcessElement}. */
private RecordContext currentProcessingContext;

public AbstractAsyncStateStreamOperatorV2(
Expand Down Expand Up @@ -182,20 +183,18 @@ public final void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> proc
@Override
@SuppressWarnings("unchecked")
public <K> void asyncProcessWithKey(K key, ThrowingRunnable<Exception> processing) {
RecordContext<K> previousContext = currentProcessingContext;

RecordContext<K> oldContext = asyncExecutionController.getCurrentContext();
// build a context and switch to the new context
currentProcessingContext = asyncExecutionController.buildContext(null, key, true);
currentProcessingContext.retain();
asyncExecutionController.setCurrentContext(currentProcessingContext);
RecordContext<K> newContext = asyncExecutionController.buildContext(null, key, true);
newContext.retain();
asyncExecutionController.setCurrentContext(newContext);
// Same logic as RECORD_ORDER, since FIRST_STATE_ORDER is problematic when the call's key
// pass the same key in.
preserveRecordOrderAndProcess(processing);
postProcessElement();
newContext.release();

// switch to original context
asyncExecutionController.setCurrentContext(previousContext);
currentProcessingContext = previousContext;
asyncExecutionController.setCurrentContext(oldContext);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.flink.util.Disposable;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;

import java.io.Closeable;

Expand Down Expand Up @@ -110,7 +111,7 @@ <N, S extends InternalKeyedState, SV> S createStateInternal(

/** By default, a state backend does nothing when a key is switched in async processing. */
@Override
default void switchContext(RecordContext<K> context) {}
default void switchContext(@Nullable RecordContext<K> context) {}

// TODO remove this once heap-based timers are working with ForSt incremental snapshots!
/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import org.apache.flink.runtime.state.v2.internal.InternalKeyedState;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;

import java.io.IOException;
import java.util.concurrent.RunnableFuture;
Expand Down Expand Up @@ -119,8 +120,10 @@ public KeyGroupRange getKeyGroupRange() {
}

@Override
public void switchContext(RecordContext<K> context) {
keyedStateBackend.setCurrentKeyAndKeyGroup(context.getKey(), context.getKeyGroup());
public void switchContext(@Nullable RecordContext<K> context) {
if (context != null) {
keyedStateBackend.setCurrentKeyAndKeyGroup(context.getKey(), context.getKeyGroup());
}
}

@Override
Expand Down

0 comments on commit b582489

Please sign in to comment.