Skip to content

Commit

Permalink
[FLINK-xxx1][Runtime] Declaring processing with async state processing
Browse files Browse the repository at this point in the history
  • Loading branch information
Zakelly committed May 10, 2024
1 parent 4fe66e0 commit a3a7988
Show file tree
Hide file tree
Showing 14 changed files with 664 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,16 @@
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.operators.asyncprocessing.declare.DeclarativeProcessingInput;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
import org.apache.flink.util.function.ThrowingConsumer;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;

import static org.apache.flink.util.Preconditions.checkArgument;
Expand All @@ -41,6 +45,9 @@
@Experimental
public abstract class AbstractInput<IN, OUT>
implements Input<IN>, KeyContextHandler, AsyncStateProcessing {

private static final Logger LOG = LoggerFactory.getLogger(AbstractInput.class);

/**
* {@code KeySelector} for extracting a key from an element being processed. This is used to
* scope keyed state to a key. This is null if the operator is not a keyed operator.
Expand Down Expand Up @@ -101,10 +108,22 @@ public final boolean isAsyncStateProcessingEnabled() {

@Internal
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public final ThrowingConsumer<StreamRecord<IN>, Exception> getRecordProcessor(int inputId) {
return AsyncStateProcessing.makeRecordProcessor(
(AsyncStateProcessingOperator) owner,
(KeySelector) stateKeySelector,
this::processElement);
if (this instanceof DeclarativeProcessingInput) {
LOG.info("declareProcess is overridden, build process is invoked.");
return AsyncStateProcessing.<IN>makeRecordProcessor(
(AsyncStateProcessingOperator) owner,
(KeySelector) stateKeySelector,
((AsyncStateProcessingOperator) owner)
.getDeclarationManager()
.<IN>buildProcess(
((DeclarativeProcessingInput<IN>) this)::declareProcess));
} else {
return AsyncStateProcessing.<IN>makeRecordProcessor(
(AsyncStateProcessingOperator) owner,
(KeySelector) stateKeySelector,
this::processElement);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,11 +39,17 @@
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
import org.apache.flink.streaming.api.operators.Triggerable;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.declare.DeclarationManager;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.declare.DeclarativeProcessingInput;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.declare.DeclarativeProcessingTwoInputOperator;
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;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;

Expand All @@ -56,13 +62,17 @@
@SuppressWarnings("rawtypes")
public abstract class AbstractAsyncStateStreamOperator<OUT> extends AbstractStreamOperator<OUT>
implements AsyncStateProcessingOperator {
private static final Logger LOG =
LoggerFactory.getLogger(AbstractAsyncStateStreamOperator.class);

private AsyncExecutionController asyncExecutionController;

private RecordContext currentProcessingContext;

private Environment environment;

private DeclarationManager declarationManager;

/** Initialize necessary state components for {@link AbstractStreamOperator}. */
@Override
public void initializeState(StreamTaskStateInitializer streamTaskStateManager)
Expand Down Expand Up @@ -95,6 +105,7 @@ public void initializeState(StreamTaskStateInitializer streamTaskStateManager)
throw new UnsupportedOperationException(
"Current State Backend doesn't support async access, AsyncExecutionController could not work");
}
this.declarationManager = new DeclarationManager();
}

private void handleAsyncStateException(String message, Throwable exception) {
Expand Down Expand Up @@ -143,6 +154,53 @@ public final void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> proc
asyncExecutionController.syncPointRequestWithCallback(processing);
}

@Override
public final DeclarationManager getDeclarationManager() {
return declarationManager;
}

// --------------------------- Declaration part ----------------------------------------
/** The ordinal identifying which process is declaring. */
enum DeclareProcessOrdinal {
INPUT1,
INPUT2,
ONLY_ONE
}

@SuppressWarnings("unchecked")
private <T> ThrowingConsumer<StreamRecord<T>, Exception> declareOverrideOrDefault(
DeclareProcessOrdinal processOrdinal,
ThrowingConsumer<StreamRecord<T>, Exception> defaultProcess) {
switch (processOrdinal) {
case INPUT1:
if (this instanceof DeclarativeProcessingTwoInputOperator) {
LOG.info("declareProcess1 is provided, build process is invoked.");
return declarationManager.buildProcess(
((DeclarativeProcessingTwoInputOperator<T, ?, ?>) this)
::declareProcess1);
}
break;
case INPUT2:
if (this instanceof DeclarativeProcessingTwoInputOperator) {
LOG.info("declareProcess2 is provided, build process is invoked.");
return declarationManager.buildProcess(
((DeclarativeProcessingTwoInputOperator<?, T, ?>) this)
::declareProcess2);
}
break;
case ONLY_ONE:
if (this instanceof DeclarativeProcessingInput) {
LOG.info("declareProcess is provided, build process is invoked.");
return declarationManager.buildProcess(
((DeclarativeProcessingInput<T>) this)::declareProcess);
}
break;
default:
break;
}
return defaultProcess;
}

@Override
@SuppressWarnings("unchecked")
public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor(int inputId) {
Expand All @@ -151,21 +209,28 @@ public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor
if (this instanceof TwoInputStreamOperator) {
switch (inputId) {
case 1:
return AsyncStateProcessing.<T>makeRecordProcessor(
return AsyncStateProcessing.makeRecordProcessor(
this,
(KeySelector) stateKeySelector1,
((TwoInputStreamOperator) this)::processElement1);
declareOverrideOrDefault(
DeclareProcessOrdinal.INPUT1,
((TwoInputStreamOperator) this)::processElement1));
case 2:
return AsyncStateProcessing.<T>makeRecordProcessor(
return AsyncStateProcessing.makeRecordProcessor(
this,
(KeySelector) stateKeySelector2,
((TwoInputStreamOperator) this)::processElement2);
declareOverrideOrDefault(
DeclareProcessOrdinal.INPUT2,
((TwoInputStreamOperator) this)::processElement2));
default:
break;
}
} else if (this instanceof Input && inputId == 1) {
return AsyncStateProcessing.<T>makeRecordProcessor(
this, (KeySelector) stateKeySelector1, ((Input) this)::processElement);
return AsyncStateProcessing.makeRecordProcessor(
this,
(KeySelector) stateKeySelector1,
declareOverrideOrDefault(
DeclareProcessOrdinal.ONLY_ONE, ((Input) this)::processElement));
}
throw new IllegalArgumentException(
String.format(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
import org.apache.flink.streaming.api.operators.Triggerable;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.declare.DeclarationManager;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.function.ThrowingConsumer;
import org.apache.flink.util.function.ThrowingRunnable;
Expand All @@ -58,10 +59,13 @@ public abstract class AbstractAsyncStateStreamOperatorV2<OUT> extends AbstractSt

private RecordContext currentProcessingContext;

private final DeclarationManager declarationManager;

public AbstractAsyncStateStreamOperatorV2(
StreamOperatorParameters<OUT> parameters, int numberOfInputs) {
super(parameters, numberOfInputs);
this.environment = parameters.getContainingTask().getEnvironment();
this.declarationManager = new DeclarationManager();
}

/** Initialize necessary state components for {@link AbstractStreamOperatorV2}. */
Expand Down Expand Up @@ -140,6 +144,11 @@ public final void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> proc
asyncExecutionController.syncPointRequestWithCallback(processing);
}

@Override
public final DeclarationManager getDeclarationManager() {
return declarationManager;
}

@Override
public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor(int inputId) {
// The real logic should be in First/SecondInputOfTwoInput#getRecordProcessor.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.declare.DeclarationManager;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.function.ThrowingRunnable;

Expand Down Expand Up @@ -53,4 +54,7 @@ <T> void setAsyncKeyedContextElement(StreamRecord<T> record, KeySelector<T, ?> k
* @param processing the record processing logic.
*/
void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> processing);

/** Get the declaration manager for user-logic declaring. */
DeclarationManager getDeclarationManager();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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.declare;

import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.function.Function;

/** A context to declare parts of process in user-defined function/operator. */
public class DeclarationContext {

private final DeclarationManager manager;

DeclarationContext(DeclarationManager manager) {
this.manager = manager;
}

// ------------- Declaring Callback part ----------------

/** Declare a callback with a name. */
public <T> NamedConsumer<T> declare(String name, Consumer<T> callback)
throws DeclarationException {
return manager.register(new NamedConsumer<>(name, callback));
}

/** Declare a callback with a name. */
public <T, V> NamedFunction<T, V> declare(String name, Function<T, V> callback)
throws DeclarationException {
return manager.register(new NamedFunction<>(name, callback));
}

/** Declare a callback with a name. */
public <T, U, V> NamedBiFunction<T, U, V> declare(String name, BiFunction<T, U, V> callback)
throws DeclarationException {
return manager.register(new NamedBiFunction<>(name, callback));
}

/** Declare a callback with an automatically assigned name. */
public <T> NamedConsumer<T> declare(Consumer<T> callback) throws DeclarationException {
return declare(manager.nextAssignedName(), callback);
}

/** Declare a callback with an automatically assigned name. */
public <T, V> NamedFunction<T, V> declare(Function<T, V> callback) throws DeclarationException {
return declare(manager.nextAssignedName(), callback);
}

/** Declare a callback with an automatically assigned name. */
public <T, U, V> NamedBiFunction<T, U, V> declare(BiFunction<T, U, V> callback)
throws DeclarationException {
return declare(manager.nextAssignedName(), callback);
}

DeclarationManager getManager() {
return manager;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.declare;

/** Exception thrown when something wrong with declaration happens. */
public class DeclarationException extends Exception {

DeclarationException(String message) {
super(message);
}
}
Loading

0 comments on commit a3a7988

Please sign in to comment.