Skip to content

Commit

Permalink
[FLINK-35025][Runtime/State] Introduce element order for async state …
Browse files Browse the repository at this point in the history
…processing
  • Loading branch information
Zakelly committed Apr 17, 2024
1 parent 21ca3ec commit af47deb
Show file tree
Hide file tree
Showing 10 changed files with 409 additions and 38 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,9 @@
import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.api.common.state.v2.State;
import org.apache.flink.core.state.InternalStateFuture;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.function.ThrowingRunnable;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -248,4 +250,23 @@ private void seizeCapacity() {
setCurrentContext(storedContext);
inFlightRecordNum.incrementAndGet();
}

/**
* A helper to request a {@link StateRequestType#SYNC_POINT} and run a callback if it finishes
* (once the record is not blocked).
*
* @param callback the callback to run if it finishes (once the record is not blocked).
*/
public void syncPointRequestWithCallback(ThrowingRunnable<Exception> callback) {
handleRequest(null, StateRequestType.SYNC_POINT, null)
.thenAccept(
v -> {
try {
callback.run();
} catch (Exception e) {
// TODO: Properly handle the exception and fail the entire job.
throw new FlinkRuntimeException("Unexpected runtime exception", e);
}
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,11 @@ public StateRequestBuffer() {
}

void enqueueToActive(StateRequest<K, ?, ?> request) {
activeQueue.add(request);
if (request.getRequestType() == StateRequestType.SYNC_POINT) {
request.getFuture().complete(null);
} else {
activeQueue.add(request);
}
}

void enqueueToBlocking(StateRequest<K, ?, ?> request) {
Expand All @@ -83,7 +87,7 @@ RecordContext<K> tryActivateOneByKey(K key) {
}

StateRequest<K, ?, ?> stateRequest = blockingQueue.get(key).removeFirst();
activeQueue.add(stateRequest);
enqueueToActive(stateRequest);
if (blockingQueue.get(key).isEmpty()) {
blockingQueue.remove(key);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -296,6 +296,50 @@ void testInFlightRecordControl() {
}
}

@Test
public void testSyncPoint() {
AtomicInteger counter = new AtomicInteger(0);

// Test the sync point processing without a key occupied.
RecordContext<String> recordContext = aec.buildContext("record", "key");
aec.setCurrentContext(recordContext);
recordContext.retain();
aec.syncPointRequestWithCallback(counter::incrementAndGet);
assertThat(counter.get()).isEqualTo(1);
assertThat(recordContext.getReferenceCount()).isEqualTo(1);
assertThat(aec.stateRequestsBuffer.activeQueueSize()).isEqualTo(0);
assertThat(aec.stateRequestsBuffer.blockingQueueSize()).isEqualTo(0);
assertThat(aec.keyAccountingUnit.occupiedCount()).isEqualTo(1);
recordContext.release();
assertThat(aec.keyAccountingUnit.occupiedCount()).isEqualTo(0);

counter.set(0);
// Test the sync point processing with a key occupied.
RecordContext<String> recordContext1 = aec.buildContext("record1", "occupied");
aec.setCurrentContext(recordContext1);
userCode.run();

RecordContext<String> recordContext2 = aec.buildContext("record2", "occupied");
aec.setCurrentContext(recordContext2);
aec.syncPointRequestWithCallback(counter::incrementAndGet);
recordContext2.retain();
assertThat(counter.get()).isEqualTo(0);
assertThat(recordContext2.getReferenceCount()).isGreaterThan(1);
assertThat(aec.stateRequestsBuffer.activeQueueSize()).isEqualTo(1);
assertThat(aec.stateRequestsBuffer.blockingQueueSize()).isEqualTo(1);
aec.triggerIfNeeded(true);
assertThat(counter.get()).isEqualTo(0);
assertThat(recordContext2.getReferenceCount()).isGreaterThan(1);
assertThat(aec.stateRequestsBuffer.activeQueueSize()).isEqualTo(1);
assertThat(aec.stateRequestsBuffer.blockingQueueSize()).isEqualTo(1);
aec.triggerIfNeeded(true);
assertThat(counter.get()).isEqualTo(1);
assertThat(recordContext2.getReferenceCount()).isEqualTo(1);
assertThat(aec.stateRequestsBuffer.activeQueueSize()).isEqualTo(0);
assertThat(aec.stateRequestsBuffer.blockingQueueSize()).isEqualTo(0);
recordContext2.release();
}

/** Simulate the underlying state that is actually used to execute the request. */
static class TestUnderlyingState {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.util.function.ThrowingConsumer;
import org.apache.flink.util.function.ThrowingRunnable;

/**
* This operator is an abstract class that give the {@link AbstractStreamOperator} the ability to
Expand Down Expand Up @@ -65,6 +66,11 @@ public boolean isAsyncStateProcessingEnabled() {
return true;
}

@Override
public ElementOrder getElementOrder() {
return ElementOrder.RECORD_ORDER;
}

@Override
@SuppressWarnings("unchecked")
public final <T> void setAsyncKeyedContextElement(
Expand All @@ -91,6 +97,12 @@ public final void postProcessElement() {
currentProcessingContext.release();
}

@Override
@SuppressWarnings("unchecked")
public final void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> processing) {
asyncExecutionController.syncPointRequestWithCallback(processing);
}

@Override
@SuppressWarnings("unchecked")
public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor(int inputId) {
Expand All @@ -99,26 +111,21 @@ public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor
if (this instanceof TwoInputStreamOperator) {
switch (inputId) {
case 1:
return record -> {
setAsyncKeyedContextElement(record, (KeySelector<T, ?>) stateKeySelector1);
((TwoInputStreamOperator) this).processElement1(record);
postProcessElement();
};
return AsyncStateProcessing.<T>makeRecordProcessor(
this,
(KeySelector) stateKeySelector1,
((TwoInputStreamOperator) this)::processElement1);
case 2:
return record -> {
setAsyncKeyedContextElement(record, (KeySelector<T, ?>) stateKeySelector2);
((TwoInputStreamOperator) this).processElement2(record);
postProcessElement();
};
return AsyncStateProcessing.<T>makeRecordProcessor(
this,
(KeySelector) stateKeySelector2,
((TwoInputStreamOperator) this)::processElement2);
default:
break;
}
} else if (this instanceof Input && inputId == 1) {
return record -> {
setAsyncKeyedContextElement(record, (KeySelector<T, ?>) stateKeySelector1);
((Input) this).processElement(record);
postProcessElement();
};
return AsyncStateProcessing.<T>makeRecordProcessor(
this, (KeySelector) stateKeySelector1, ((Input) this)::processElement);
}
throw new IllegalArgumentException(
String.format(
Expand All @@ -130,4 +137,9 @@ public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor
AsyncExecutionController<?> getAsyncExecutionController() {
return asyncExecutionController;
}

@VisibleForTesting
RecordContext getCurrentProcessingContext() {
return currentProcessingContext;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.function.ThrowingConsumer;
import org.apache.flink.util.function.ThrowingRunnable;

/**
* This operator is an abstract class that give the {@link AbstractStreamOperatorV2} the ability to
Expand Down Expand Up @@ -67,6 +68,11 @@ public boolean isAsyncStateProcessingEnabled() {
return true;
}

@Override
public ElementOrder getElementOrder() {
return ElementOrder.RECORD_ORDER;
}

@Override
@SuppressWarnings("unchecked")
public final <T> void setAsyncKeyedContextElement(
Expand All @@ -93,6 +99,12 @@ public final void postProcessElement() {
currentProcessingContext.release();
}

@Override
@SuppressWarnings("unchecked")
public final void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> processing) {
asyncExecutionController.syncPointRequestWithCallback(processing);
}

@Override
public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor(int inputId) {
// The real logic should be in First/SecondInputOfTwoInput#getRecordProcessor.
Expand All @@ -105,4 +117,9 @@ public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor
AsyncExecutionController<?> getAsyncExecutionController() {
return asyncExecutionController;
}

@VisibleForTesting
RecordContext getCurrentProcessingContext() {
return currentProcessingContext;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,10 +56,23 @@ static <T> ThrowingConsumer<StreamRecord<T>, Exception> makeRecordProcessor(
AsyncStateProcessingOperator asyncOperator,
KeySelector<T, ?> keySelector,
ThrowingConsumer<StreamRecord<T>, Exception> processor) {
return (record) -> {
asyncOperator.setAsyncKeyedContextElement(record, keySelector);
processor.accept(record);
asyncOperator.postProcessElement();
};
switch (asyncOperator.getElementOrder()) {
case RECORD_ORDER:
return (record) -> {
asyncOperator.setAsyncKeyedContextElement(record, keySelector);
asyncOperator.preserveRecordOrderAndProcess(() -> processor.accept(record));
asyncOperator.postProcessElement();
};
case FIRST_STATE_ORDER:
return (record) -> {
asyncOperator.setAsyncKeyedContextElement(record, keySelector);
processor.accept(record);
asyncOperator.postProcessElement();
};
default:
throw new UnsupportedOperationException(
"Unknown element order for async processing:"
+ asyncOperator.getElementOrder());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,17 @@

import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.function.ThrowingRunnable;

/**
* A more detailed interface based on {@link AsyncStateProcessing}, which gives the essential
* methods for an operator to perform async state processing.
*/
public interface AsyncStateProcessingOperator extends AsyncStateProcessing {

/** Get the {@link ElementOrder} of this operator. */
ElementOrder getElementOrder();

/**
* Set key context for async state processing.
*
Expand All @@ -39,4 +43,12 @@ <T> void setAsyncKeyedContextElement(StreamRecord<T> record, KeySelector<T, ?> k

/** A callback that will be triggered after an element finishes {@code processElement}. */
void postProcessElement();

/**
* Check the order of same-key record, and then process the record. Mainly used when the {@link
* #getElementOrder()} returns {@link ElementOrder#RECORD_ORDER}.
*
* @param processing the record processing logic.
*/
void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> processing);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.streaming.runtime.operators.asyncprocessing;

import org.apache.flink.annotation.Internal;

/**
* This enum defines the element order of being processed. Only the elements with the same key
* should be considered here. We should keep this internal and away from API module for now, until
* we could see the concrete need for {@link #FIRST_STATE_ORDER} from average users.
*/
@Internal
public enum ElementOrder {
/**
* Treat the record processing as a whole, meaning that any {@code processElement} call for the
* elements with same key should follow the order of record arrival AND no parallel run is
* allowed.
*/
RECORD_ORDER,

/**
* The {@code processElement} call will be invoked on record arrival, but may be blocked at the
* first state accessing if there is a preceding same-key record under processing.
*/
FIRST_STATE_ORDER,
}
Loading

0 comments on commit af47deb

Please sign in to comment.