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-35025][Runtime/State] Abstract stream operators for async stat…
…e processing
- Loading branch information
Showing
10 changed files
with
595 additions
and
4 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
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
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
127 changes: 127 additions & 0 deletions
127
...e/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.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,127 @@ | ||
/* | ||
* 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.annotation.VisibleForTesting; | ||
import org.apache.flink.api.common.operators.MailboxExecutor; | ||
import org.apache.flink.api.common.typeutils.TypeSerializer; | ||
import org.apache.flink.api.java.functions.KeySelector; | ||
import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController; | ||
import org.apache.flink.runtime.asyncprocessing.RecordContext; | ||
import org.apache.flink.streaming.api.graph.StreamConfig; | ||
import org.apache.flink.streaming.api.operators.AbstractStreamOperator; | ||
import org.apache.flink.streaming.api.operators.Output; | ||
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; | ||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; | ||
import org.apache.flink.streaming.runtime.tasks.StreamTask; | ||
import org.apache.flink.util.function.ThrowingConsumer; | ||
|
||
import java.lang.reflect.ParameterizedType; | ||
|
||
/** | ||
* This operator is an abstract class that give the {@link AbstractStreamOperator} the ability to | ||
* perform {@link AsyncStateProcessing}. The aim is to make any subclass of {@link | ||
* AbstractStreamOperator} could manipulate async state with only a change of base class. | ||
*/ | ||
@Internal | ||
@SuppressWarnings("rawtypes") | ||
public abstract class AbstractAsyncStateStreamOperator<OUT> extends AbstractStreamOperator<OUT> | ||
implements AsyncStateProcessing { | ||
|
||
private AsyncExecutionController asyncExecutionController; | ||
|
||
private RecordContext lastProcessContext; | ||
|
||
/** Initialize necessary state components for {@link AbstractStreamOperator}. */ | ||
@Override | ||
public void setup( | ||
StreamTask<?, ?> containingTask, | ||
StreamConfig config, | ||
Output<StreamRecord<OUT>> output) { | ||
super.setup(containingTask, config, output); | ||
// TODO: properly read config and setup | ||
final MailboxExecutor mailboxExecutor = | ||
containingTask.getEnvironment().getMainMailboxExecutor(); | ||
this.asyncExecutionController = | ||
new AsyncExecutionController(getTypeClassOfKey(), mailboxExecutor, null); | ||
} | ||
|
||
private Class<?> getTypeClassOfKey() { | ||
final TypeSerializer<?> keySerializer = | ||
config.getStateKeySerializer(getUserCodeClassloader()); | ||
return (Class) | ||
((ParameterizedType) keySerializer.getClass().getGenericSuperclass()) | ||
.getActualTypeArguments()[0]; | ||
} | ||
|
||
@Override | ||
public final boolean isAsyncStateProcessingEnabled() { | ||
// TODO: Read from config | ||
return true; | ||
} | ||
|
||
@Override | ||
@SuppressWarnings("unchecked") | ||
public final <T> void setAsyncKeyedContextElement( | ||
StreamRecord<T> record, KeySelector<T, ?> keySelector) throws Exception { | ||
lastProcessContext = | ||
asyncExecutionController.buildContext( | ||
record.getValue(), keySelector.getKey(record.getValue())); | ||
lastProcessContext.retain(); | ||
asyncExecutionController.setCurrentContext(lastProcessContext); | ||
} | ||
|
||
@Override | ||
public final void postProcessElement() { | ||
lastProcessContext.release(); | ||
} | ||
|
||
@Override | ||
@SuppressWarnings("unchecked") | ||
public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor( | ||
InputOrdinal input) { | ||
// Ideally, only TwoStreamInputOperator will invoke here. | ||
if (this instanceof TwoInputStreamOperator) { | ||
switch (input) { | ||
case FIRST: | ||
return record -> { | ||
setAsyncKeyedContextElement(record, (KeySelector<T, ?>) stateKeySelector1); | ||
((TwoInputStreamOperator) this).processElement1(record); | ||
postProcessElement(); | ||
}; | ||
case SECOND: | ||
return record -> { | ||
setAsyncKeyedContextElement(record, (KeySelector<T, ?>) stateKeySelector2); | ||
((TwoInputStreamOperator) this).processElement2(record); | ||
postProcessElement(); | ||
}; | ||
default: | ||
throw new IllegalArgumentException("Unsupported input ordinal."); | ||
} | ||
} | ||
throw new UnsupportedOperationException( | ||
"Unsupported operator type for async processing:" + getClass().getName()); | ||
} | ||
|
||
@VisibleForTesting | ||
AsyncExecutionController<?> getAsyncExecutionController() { | ||
return asyncExecutionController; | ||
} | ||
} |
110 changes: 110 additions & 0 deletions
110
...flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperatorV2.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,110 @@ | ||
/* | ||
* 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.annotation.VisibleForTesting; | ||
import org.apache.flink.api.common.operators.MailboxExecutor; | ||
import org.apache.flink.api.common.typeutils.TypeSerializer; | ||
import org.apache.flink.api.java.functions.KeySelector; | ||
import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController; | ||
import org.apache.flink.runtime.asyncprocessing.RecordContext; | ||
import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; | ||
import org.apache.flink.streaming.api.operators.StreamOperatorParameters; | ||
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; | ||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; | ||
import org.apache.flink.util.function.ThrowingConsumer; | ||
|
||
import java.lang.reflect.ParameterizedType; | ||
|
||
/** | ||
* This operator is an abstract class that give the {@link AbstractStreamOperatorV2} the ability to | ||
* perform {@link AsyncStateProcessing}. The aim is to make any subclass of {@link | ||
* AbstractStreamOperatorV2} could manipulate async state with only a change of base class. | ||
*/ | ||
@Internal | ||
@SuppressWarnings("rawtypes") | ||
public abstract class AbstractAsyncStateStreamOperatorV2<OUT> extends AbstractStreamOperatorV2<OUT> | ||
implements AsyncStateProcessing { | ||
|
||
private AsyncExecutionController asyncExecutionController; | ||
|
||
private RecordContext lastProcessContext; | ||
|
||
public AbstractAsyncStateStreamOperatorV2( | ||
StreamOperatorParameters<OUT> parameters, int numberOfInputs) { | ||
super(parameters, numberOfInputs); | ||
} | ||
|
||
/** Initialize necessary state components for {@link AbstractStreamOperatorV2}. */ | ||
@Override | ||
public final void initializeState(StreamTaskStateInitializer streamTaskStateManager) | ||
throws Exception { | ||
super.initializeState(streamTaskStateManager); | ||
// TODO: Read config and properly set. | ||
final MailboxExecutor mailboxExecutor = | ||
getRuntimeContext().getTaskEnvironment().getMainMailboxExecutor(); | ||
this.asyncExecutionController = | ||
new AsyncExecutionController(getTypeClassOfKey(), mailboxExecutor, null); | ||
} | ||
|
||
private Class<?> getTypeClassOfKey() { | ||
final TypeSerializer<?> keySerializer = | ||
config.getStateKeySerializer(getUserCodeClassloader()); | ||
return (Class) | ||
((ParameterizedType) keySerializer.getClass().getGenericSuperclass()) | ||
.getActualTypeArguments()[0]; | ||
} | ||
|
||
@Override | ||
public final boolean isAsyncStateProcessingEnabled() { | ||
// TODO: Read from config | ||
return true; | ||
} | ||
|
||
@Override | ||
@SuppressWarnings("unchecked") | ||
public final <T> void setAsyncKeyedContextElement( | ||
StreamRecord<T> record, KeySelector<T, ?> keySelector) throws Exception { | ||
lastProcessContext = | ||
asyncExecutionController.buildContext( | ||
record.getValue(), keySelector.getKey(record.getValue())); | ||
lastProcessContext.retain(); | ||
asyncExecutionController.setCurrentContext(lastProcessContext); | ||
} | ||
|
||
@Override | ||
public final void postProcessElement() { | ||
lastProcessContext.release(); | ||
} | ||
|
||
@Override | ||
public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor( | ||
InputOrdinal input) { | ||
// The real logic should be in AbstractInput#getRecordProcessor. | ||
throw new UnsupportedOperationException( | ||
"Never getRecordProcessor from AbstractAsyncStateStreamOperatorV2," | ||
+ " since this part is handled by the Input."); | ||
} | ||
|
||
@VisibleForTesting | ||
AsyncExecutionController<?> getAsyncExecutionController() { | ||
return asyncExecutionController; | ||
} | ||
} |
Oops, something went wrong.