forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-19485] Add TransformationTranslator framework and OneInputTran…
…slator Adds the TransformationTranslator, the framework based on which Transformations are going to be translated into their runtime implementations depending on if they are to be executed in BATCH or STREAMING and an example using the OneInputTransformation. Now the user will have to write a Translator for each Transformation and register the Translator with the StreamGraphGenerator by putting it in the `translatorMap`. Translators must: 1) have a 0-arg constructor 2) be stateless The boilerplate transformation translation code is moved from the StreamGraphGenerator to the SimpleTransformationTranslator. This is going to be the base translator for all existing Transformations and all future ones which have a 1-to-1 mapping between transformation and operation.
- Loading branch information
Showing
5 changed files
with
420 additions
and
48 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
126 changes: 126 additions & 0 deletions
126
...va/src/main/java/org/apache/flink/streaming/api/graph/SimpleTransformationTranslator.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,126 @@ | ||
/* | ||
* 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.api.graph; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
import org.apache.flink.api.dag.Transformation; | ||
import org.apache.flink.streaming.api.transformations.PhysicalTransformation; | ||
|
||
import java.util.Collection; | ||
|
||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
|
||
/** | ||
* A base class for all {@link TransformationTranslator TransformationTranslators} who translate | ||
* {@link Transformation Transformations} that have a single operator in their runtime implementation. | ||
* These include most of the currently supported operations. | ||
* | ||
* @param <OUT> The type of the output elements of the transformation being translated. | ||
* @param <T> The type of transformation being translated. | ||
*/ | ||
@Internal | ||
public abstract class SimpleTransformationTranslator<OUT, T extends Transformation<OUT>> | ||
implements TransformationTranslator<OUT, T> { | ||
|
||
@Override | ||
public Collection<Integer> translateForBatch(final T transformation, final Context context) { | ||
checkNotNull(transformation); | ||
checkNotNull(context); | ||
|
||
final Collection<Integer> transformedIds = | ||
translateForBatchInternal(transformation, context); | ||
configure(transformation, context); | ||
|
||
return transformedIds; | ||
} | ||
|
||
@Override | ||
public Collection<Integer> translateForStreaming(final T transformation, final Context context) { | ||
checkNotNull(transformation); | ||
checkNotNull(context); | ||
|
||
final Collection<Integer> transformedIds = | ||
translateForStreamingInternal(transformation, context); | ||
configure(transformation, context); | ||
|
||
return transformedIds; | ||
} | ||
|
||
/** | ||
* Translates a given {@link Transformation} to its runtime implementation for BATCH-style execution. | ||
* | ||
* @param transformation The transformation to be translated. | ||
* @param context The translation context. | ||
* @return The ids of the "last" {@link StreamNode StreamNodes} in the transformation graph corresponding | ||
* to this transformation. These will be the nodes that a potential following transformation will need to | ||
* connect to. | ||
*/ | ||
protected abstract Collection<Integer> translateForBatchInternal(final T transformation, final Context context); | ||
|
||
/** | ||
* Translates a given {@link Transformation} to its runtime implementation for STREAMING-style execution. | ||
* | ||
* @param transformation The transformation to be translated. | ||
* @param context The translation context. | ||
* @return The ids of the "last" {@link StreamNode StreamNodes} in the transformation graph corresponding | ||
* to this transformation. These will be the nodes that a potential following transformation will need to | ||
* connect to. | ||
*/ | ||
protected abstract Collection<Integer> translateForStreamingInternal(final T transformation, final Context context); | ||
|
||
private void configure(final T transformation, final Context context) { | ||
final StreamGraph streamGraph = context.getStreamGraph(); | ||
final int transformationId = transformation.getId(); | ||
|
||
if (transformation.getBufferTimeout() >= 0) { | ||
streamGraph.setBufferTimeout(transformationId, transformation.getBufferTimeout()); | ||
} else { | ||
streamGraph.setBufferTimeout(transformationId, context.getDefaultBufferTimeout()); | ||
} | ||
|
||
if (transformation.getUid() != null) { | ||
streamGraph.setTransformationUID(transformationId, transformation.getUid()); | ||
} | ||
if (transformation.getUserProvidedNodeHash() != null) { | ||
streamGraph.setTransformationUserHash(transformationId, transformation.getUserProvidedNodeHash()); | ||
} | ||
|
||
if (!streamGraph.getExecutionConfig().hasAutoGeneratedUIDsEnabled()) { | ||
if (transformation instanceof PhysicalTransformation | ||
&& transformation.getUserProvidedNodeHash() == null | ||
&& transformation.getUid() == null) { | ||
throw new IllegalStateException("Auto generated UIDs have been disabled " + | ||
"but no UID or hash has been assigned to operator " + transformation.getName()); | ||
} | ||
} | ||
|
||
if (transformation.getMinResources() != null && transformation.getPreferredResources() != null) { | ||
streamGraph.setResources(transformationId, transformation.getMinResources(), transformation.getPreferredResources()); | ||
} | ||
|
||
final StreamNode streamNode = streamGraph.getStreamNode(transformationId); | ||
if (streamNode != null | ||
&& streamNode.getManagedMemoryOperatorScopeUseCaseWeights().isEmpty() | ||
&& streamNode.getManagedMemorySlotScopeUseCases().isEmpty()) { | ||
streamNode.setManagedMemoryUseCaseWeights( | ||
transformation.getManagedMemoryOperatorScopeUseCaseWeights(), | ||
transformation.getManagedMemorySlotScopeUseCases()); | ||
} | ||
} | ||
} |
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
Oops, something went wrong.