Skip to content

Commit

Permalink
[FLINK-35025][Runtime/State] Split AsyncStateProcessing and introdu…
Browse files Browse the repository at this point in the history
…ce element order for state processing
  • Loading branch information
Zakelly committed Apr 16, 2024
1 parent e99640a commit 054d8e5
Show file tree
Hide file tree
Showing 13 changed files with 429 additions and 112 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,6 +60,9 @@ public StateRequestBuffer() {
}

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

Expand All @@ -83,7 +86,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 @@ -23,6 +23,7 @@
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncStateProcessing;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncStateProcessingOperator;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
Expand Down Expand Up @@ -94,32 +95,16 @@ public boolean hasKeyContext() {
@Internal
@Override
public final boolean isAsyncStateProcessingEnabled() {
return (owner instanceof AsyncStateProcessing)
&& ((AsyncStateProcessing) owner).isAsyncStateProcessingEnabled();
return (owner instanceof AsyncStateProcessingOperator)
&& ((AsyncStateProcessingOperator) owner).isAsyncStateProcessingEnabled();
}

@Internal
@Override
@SuppressWarnings("unchecked")
public final <T> void setAsyncKeyedContextElement(
StreamRecord<T> record, KeySelector<T, ?> keySelector) throws Exception {
((AsyncStateProcessing) owner).setAsyncKeyedContextElement(record, keySelector);
}

@Internal
@Override
public final void postProcessElement() {
((AsyncStateProcessing) owner).postProcessElement();
}

@Internal
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public final ThrowingConsumer<StreamRecord, Exception> getRecordProcessor(int inputId) {
return record -> {
setAsyncKeyedContextElement(record, stateKeySelector);
processElement(record);
postProcessElement();
};
public final ThrowingConsumer<StreamRecord<IN>, Exception> getRecordProcessor(int inputId) {
return AsyncStateProcessing.makeRecordProcessor(
(AsyncStateProcessingOperator) owner,
(KeySelector) stateKeySelector,
this::processElement);
}
}
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 All @@ -41,7 +42,7 @@
@Internal
@SuppressWarnings("rawtypes")
public abstract class AbstractAsyncStateStreamOperator<OUT> extends AbstractStreamOperator<OUT>
implements AsyncStateProcessing {
implements AsyncStateProcessingOperator {

private AsyncExecutionController asyncExecutionController;

Expand All @@ -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 All @@ -38,7 +39,7 @@
@Internal
@SuppressWarnings("rawtypes")
public abstract class AbstractAsyncStateStreamOperatorV2<OUT> extends AbstractStreamOperatorV2<OUT>
implements AsyncStateProcessing {
implements AsyncStateProcessingOperator {

private AsyncExecutionController asyncExecutionController;

Expand All @@ -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
public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor(int inputId) {
// The real logic should be in First/SecondInputOfTwoInput#getRecordProcessor.
Expand All @@ -103,4 +115,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 @@ -34,19 +34,6 @@ public interface AsyncStateProcessing {
*/
boolean isAsyncStateProcessingEnabled();

/**
* Set key context for async state processing.
*
* @param record the record.
* @param keySelector the key selector to select a key from record.
* @param <T> the type of the record.
*/
<T> void setAsyncKeyedContextElement(StreamRecord<T> record, KeySelector<T, ?> keySelector)
throws Exception;

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

/**
* Get the record processor that could process record from input, which is the only entry for
* async processing.
Expand All @@ -56,4 +43,36 @@ <T> void setAsyncKeyedContextElement(StreamRecord<T> record, KeySelector<T, ?> k
* there is multiple inputs for the instance.
*/
<T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor(int inputId);

/**
* Static method helper to make a record processor with given infos.
*
* @param asyncOperator the operator that can process state asynchronously.
* @param keySelector the key selector.
* @param processor the record processing logic.
* @return the built record processor that can returned by {@link #getRecordProcessor(int)}.
*/
static <T> ThrowingConsumer<StreamRecord<T>, Exception> makeRecordProcessor(
AsyncStateProcessingOperator asyncOperator,
KeySelector<T, ?> keySelector,
ThrowingConsumer<StreamRecord<T>, Exception> processor) {
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
@@ -0,0 +1,54 @@
/*
* 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.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.
*
* @param record the record.
* @param keySelector the key selector to select a key from record.
* @param <T> the type of the record.
*/
<T> void setAsyncKeyedContextElement(StreamRecord<T> record, KeySelector<T, ?> keySelector)
throws Exception;

/** 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 054d8e5

Please sign in to comment.