forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-35156][Runtime] Make operators of DataStream V2 integrate with…
… async state processing
- Loading branch information
Showing
3 changed files
with
182 additions
and
2 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
168 changes: 168 additions & 0 deletions
168
...link/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateUdfStreamOperator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,168 @@ | ||
/* | ||
* 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; | ||
import org.apache.flink.api.common.ExecutionConfig; | ||
import org.apache.flink.api.common.functions.DefaultOpenContext; | ||
import org.apache.flink.api.common.functions.Function; | ||
import org.apache.flink.api.common.functions.util.FunctionUtils; | ||
import org.apache.flink.api.common.state.CheckpointListener; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.runtime.state.StateInitializationContext; | ||
import org.apache.flink.runtime.state.StateSnapshotContext; | ||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; | ||
import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; | ||
import org.apache.flink.streaming.api.functions.sink.SinkFunction; | ||
import org.apache.flink.streaming.api.graph.StreamConfig; | ||
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; | ||
import org.apache.flink.streaming.api.operators.Output; | ||
import org.apache.flink.streaming.api.operators.OutputTypeConfigurable; | ||
import org.apache.flink.streaming.api.operators.UserFunctionProvider; | ||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; | ||
import org.apache.flink.streaming.runtime.tasks.StreamTask; | ||
import org.apache.flink.streaming.util.functions.StreamingFunctionUtils; | ||
|
||
import static java.util.Objects.requireNonNull; | ||
|
||
/** | ||
* This is used as the base class for operators that have a user-defined function. This class | ||
* handles the opening and closing of the user-defined functions, as part of the operator life | ||
* cycle. This class is nearly identical with {@link AbstractUdfStreamOperator}, but extending from | ||
* {@link AbstractAsyncStateStreamOperator} to integrate with asynchronous state access. Another | ||
* difference is this class is internal. | ||
* | ||
* @param <OUT> The output type of the operator | ||
* @param <F> The type of the user function | ||
*/ | ||
@Internal | ||
public abstract class AbstractAsyncStateUdfStreamOperator<OUT, F extends Function> | ||
extends AbstractAsyncStateStreamOperator<OUT> | ||
implements OutputTypeConfigurable<OUT>, UserFunctionProvider<F> { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
/** The user function. */ | ||
protected final F userFunction; | ||
|
||
public AbstractAsyncStateUdfStreamOperator(F userFunction) { | ||
this.userFunction = requireNonNull(userFunction); | ||
checkUdfCheckpointingPreconditions(); | ||
} | ||
|
||
/** | ||
* Gets the user function executed in this operator. | ||
* | ||
* @return The user function of this operator. | ||
*/ | ||
public F getUserFunction() { | ||
return userFunction; | ||
} | ||
|
||
// ------------------------------------------------------------------------ | ||
// operator life cycle | ||
// ------------------------------------------------------------------------ | ||
|
||
@Override | ||
public void setup( | ||
StreamTask<?, ?> containingTask, | ||
StreamConfig config, | ||
Output<StreamRecord<OUT>> output) { | ||
super.setup(containingTask, config, output); | ||
FunctionUtils.setFunctionRuntimeContext(userFunction, getRuntimeContext()); | ||
} | ||
|
||
@Override | ||
public void snapshotState(StateSnapshotContext context) throws Exception { | ||
super.snapshotState(context); | ||
StreamingFunctionUtils.snapshotFunctionState( | ||
context, getOperatorStateBackend(), userFunction); | ||
} | ||
|
||
@Override | ||
public void initializeState(StateInitializationContext context) throws Exception { | ||
super.initializeState(context); | ||
StreamingFunctionUtils.restoreFunctionState(context, userFunction); | ||
} | ||
|
||
@Override | ||
public void open() throws Exception { | ||
super.open(); | ||
FunctionUtils.openFunction(userFunction, DefaultOpenContext.INSTANCE); | ||
} | ||
|
||
@Override | ||
public void finish() throws Exception { | ||
super.finish(); | ||
if (userFunction instanceof SinkFunction) { | ||
((SinkFunction<?>) userFunction).finish(); | ||
} | ||
} | ||
|
||
@Override | ||
public void close() throws Exception { | ||
super.close(); | ||
FunctionUtils.closeFunction(userFunction); | ||
} | ||
|
||
// ------------------------------------------------------------------------ | ||
// checkpointing and recovery | ||
// ------------------------------------------------------------------------ | ||
|
||
@Override | ||
public void notifyCheckpointComplete(long checkpointId) throws Exception { | ||
super.notifyCheckpointComplete(checkpointId); | ||
|
||
if (userFunction instanceof CheckpointListener) { | ||
((CheckpointListener) userFunction).notifyCheckpointComplete(checkpointId); | ||
} | ||
} | ||
|
||
@Override | ||
public void notifyCheckpointAborted(long checkpointId) throws Exception { | ||
super.notifyCheckpointAborted(checkpointId); | ||
|
||
if (userFunction instanceof CheckpointListener) { | ||
((CheckpointListener) userFunction).notifyCheckpointAborted(checkpointId); | ||
} | ||
} | ||
|
||
// ------------------------------------------------------------------------ | ||
// Output type configuration | ||
// ------------------------------------------------------------------------ | ||
|
||
@Override | ||
public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) { | ||
StreamingFunctionUtils.setOutputType(userFunction, outTypeInfo, executionConfig); | ||
} | ||
|
||
// ------------------------------------------------------------------------ | ||
// Utilities | ||
// ------------------------------------------------------------------------ | ||
|
||
private void checkUdfCheckpointingPreconditions() { | ||
if (userFunction instanceof CheckpointedFunction | ||
&& userFunction instanceof ListCheckpointed) { | ||
|
||
throw new IllegalStateException( | ||
"User functions are not allowed to implement " | ||
+ "CheckpointedFunction AND ListCheckpointed."); | ||
} | ||
} | ||
} |