Skip to content

Commit

Permalink
[FLINK-19485] Add TransformationTranslator framework and OneInputTran…
Browse files Browse the repository at this point in the history
…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
kl0u committed Oct 12, 2020
1 parent a2202ee commit 0c3dfc8
Show file tree
Hide file tree
Showing 5 changed files with 420 additions and 48 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -441,12 +441,11 @@ public void testParallelismOnLimitPushDown() {
Table table = tEnv.sqlQuery("select * from hive.source_db.test_parallelism_limit_pushdown limit 1");
PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) tEnv).getPlanner();
RelNode relNode = planner.optimize(TableTestUtil.toRelNode(table));
ExecNode execNode = planner.translateToExecNodePlan(toScala(Collections.singletonList(relNode))).get(0);
@SuppressWarnings("unchecked")
Transformation transformation = (Transformation) ((Transformation) execNode.translateToPlan(planner).getInputs().get(0)).getInputs().get(0);
Assert.assertEquals(
1,
transformation.getParallelism());
ExecNode<PlannerBase, ?> execNode = (ExecNode<PlannerBase, ?>) planner.translateToExecNodePlan(
toScala(Collections.singletonList(relNode))).get(0);
Transformation<?> transformation = (execNode.translateToPlan(planner).getInputs().get(0)).getInputs().get(0);
Assert.assertEquals(1, transformation.getParallelism());
}

@Test
Expand Down
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());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.flink.streaming.api.transformations.UnionTransformation;
import org.apache.flink.streaming.api.transformations.WithBoundedness;
import org.apache.flink.streaming.runtime.io.MultipleInputSelectionHandler;
import org.apache.flink.streaming.runtime.translators.OneInputTransformationTranslator;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -128,6 +129,18 @@ public class StreamGraphGenerator {

private RuntimeExecutionMode runtimeExecutionMode = RuntimeExecutionMode.STREAMING;

private boolean shouldExecuteInBatchMode;

@SuppressWarnings("rawtypes")
private static final Map<Class<? extends Transformation>, TransformationTranslator<?, ? extends Transformation>> translatorMap;

static {
@SuppressWarnings("rawtypes")
Map<Class<? extends Transformation>, TransformationTranslator<?, ? extends Transformation>> tmp = new HashMap<>();
tmp.put(OneInputTransformation.class, new OneInputTransformationTranslator<>());
translatorMap = Collections.unmodifiableMap(tmp);
}

// This is used to assign a unique ID to iteration source/sink
protected static Integer iterationIdCounter = 0;
public static int getNewIterationNodeId() {
Expand Down Expand Up @@ -191,6 +204,7 @@ public void setSavepointRestoreSettings(SavepointRestoreSettings savepointRestor

public StreamGraph generate() {
streamGraph = new StreamGraph(executionConfig, checkpointConfig, savepointRestoreSettings);
shouldExecuteInBatchMode = shouldExecuteInBatchMode(runtimeExecutionMode);
configureStreamGraph(streamGraph);

alreadyTransformed = new HashMap<>();
Expand All @@ -217,7 +231,7 @@ private void configureStreamGraph(final StreamGraph graph) {
graph.setTimeCharacteristic(timeCharacteristic);
graph.setJobName(jobName);

if (shouldExecuteInBatchMode(runtimeExecutionMode)) {
if (shouldExecuteInBatchMode) {
graph.setAllVerticesInSameSlotSharingGroupByDefault(false);
graph.setGlobalDataExchangeMode(GlobalDataExchangeMode.POINTWISE_EDGES_PIPELINED);
graph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST);
Expand Down Expand Up @@ -277,10 +291,29 @@ private Collection<Integer> transform(Transformation<?> transform) {
// call at least once to trigger exceptions about MissingTypeInfo
transform.getOutputType();

@SuppressWarnings("unchecked")
final TransformationTranslator<?, Transformation<?>> translator =
(TransformationTranslator<?, Transformation<?>>) translatorMap.get(transform.getClass());

Collection<Integer> transformedIds;
if (translator != null) {
transformedIds = translate(translator, transform);
} else {
transformedIds = legacyTransform(transform);
}

// need this check because the iterate transformation adds itself before
// transforming the feedback edges
if (!alreadyTransformed.containsKey(transform)) {
alreadyTransformed.put(transform, transformedIds);
}

return transformedIds;
}

private Collection<Integer> legacyTransform(Transformation<?> transform) {
Collection<Integer> transformedIds;
if (transform instanceof OneInputTransformation<?, ?>) {
transformedIds = transformOneInputTransform((OneInputTransformation<?, ?>) transform);
} else if (transform instanceof TwoInputTransformation<?, ?, ?>) {
if (transform instanceof TwoInputTransformation<?, ?, ?>) {
transformedIds = transformTwoInputTransform((TwoInputTransformation<?, ?, ?>) transform);
} else if (transform instanceof AbstractMultipleInputTransformation<?>) {
transformedIds = transformMultipleInputTransform((AbstractMultipleInputTransformation<?>) transform);
Expand All @@ -304,12 +337,6 @@ private Collection<Integer> transform(Transformation<?> transform) {
throw new IllegalStateException("Unknown transformation: " + transform);
}

// need this check because the iterate transformation adds itself before
// transforming the feedback edges
if (!alreadyTransformed.containsKey(transform)) {
alreadyTransformed.put(transform, transformedIds);
}

if (transform.getBufferTimeout() >= 0) {
streamGraph.setBufferTimeout(transform.getId(), transform.getBufferTimeout());
} else {
Expand Down Expand Up @@ -645,48 +672,31 @@ private <T> Collection<Integer> transformSink(SinkTransformation<T> sink) {
return Collections.emptyList();
}

/**
* Transforms a {@code OneInputTransformation}.
*
* <p>This recursively transforms the inputs, creates a new {@code StreamNode} in the graph and
* wired the inputs to this new node.
*/
private <IN, OUT> Collection<Integer> transformOneInputTransform(OneInputTransformation<IN, OUT> transform) {
private Collection<Integer> translate(
final TransformationTranslator<?, Transformation<?>> translator,
final Transformation<?> transform) {
checkNotNull(translator);
checkNotNull(transform);

List<Collection<Integer>> allInputIds = getParentInputIds(transform.getInputs());
checkState(allInputIds.size() == 1);
Collection<Integer> inputIds = allInputIds.get(0);
final List<Collection<Integer>> allInputIds = getParentInputIds(transform.getInputs());

// the recursive call might have already transformed this
if (alreadyTransformed.containsKey(transform)) {
return alreadyTransformed.get(transform);
}

String slotSharingGroup = determineSlotSharingGroup(transform.getSlotSharingGroup(), inputIds);

streamGraph.addOperator(transform.getId(),
slotSharingGroup,
transform.getCoLocationGroupKey(),
transform.getOperatorFactory(),
transform.getInputType(),
transform.getOutputType(),
transform.getName());

if (transform.getStateKeySelector() != null) {
TypeSerializer<?> keySerializer = transform.getStateKeyType().createSerializer(executionConfig);
streamGraph.setOneInputStateKey(transform.getId(), transform.getStateKeySelector(), keySerializer);
}

int parallelism = transform.getParallelism() != ExecutionConfig.PARALLELISM_DEFAULT ?
transform.getParallelism() : executionConfig.getParallelism();
streamGraph.setParallelism(transform.getId(), parallelism);
streamGraph.setMaxParallelism(transform.getId(), transform.getMaxParallelism());
final String slotSharingGroup = determineSlotSharingGroup(
transform.getSlotSharingGroup(),
allInputIds.stream()
.flatMap(Collection::stream)
.collect(Collectors.toList()));

for (Integer inputId: inputIds) {
streamGraph.addEdge(inputId, transform.getId(), 0);
}
final TransformationTranslator.Context context = new ContextImpl(
this, streamGraph, slotSharingGroup);

return Collections.singleton(transform.getId());
return shouldExecuteInBatchMode
? translator.translateForBatch(transform, context)
: translator.translateForStreaming(transform, context);
}

/**
Expand Down Expand Up @@ -798,6 +808,8 @@ private <OUT> Collection<Integer> transformMultipleInputTransform(AbstractMultip
* Returns a list of lists containing the ids of the nodes in the transformation graph
* that correspond to the provided transformations. Each transformation may have multiple nodes.
*
* <p>Parent transformations will be translated if they are not already translated.
*
* @param parentTransformations the transformations whose node ids to return.
* @return the nodeIds per transformation or an empty list if the {@code parentTransformations} are empty.
*/
Expand Down Expand Up @@ -841,4 +853,45 @@ private String determineSlotSharingGroup(String specifiedGroup, Collection<Integ
return inputGroup == null ? DEFAULT_SLOT_SHARING_GROUP : inputGroup;
}
}

private static class ContextImpl implements TransformationTranslator.Context {

private final StreamGraphGenerator streamGraphGenerator;

private final StreamGraph streamGraph;

private final String slotSharingGroup;

public ContextImpl(
final StreamGraphGenerator streamGraphGenerator,
final StreamGraph streamGraph,
final String slotSharingGroup) {
this.streamGraphGenerator = checkNotNull(streamGraphGenerator);
this.streamGraph = checkNotNull(streamGraph);
this.slotSharingGroup = checkNotNull(slotSharingGroup);
}

@Override
public StreamGraph getStreamGraph() {
return streamGraph;
}

@Override
public Collection<Integer> getStreamNodeIds(final Transformation<?> transformation) {
checkNotNull(transformation);
final Collection<Integer> ids = streamGraphGenerator.alreadyTransformed.get(transformation);
checkState(ids != null, "Parent transformation \"" + transformation + "\" has not been transformed.");
return ids;
}

@Override
public String getSlotSharingGroup() {
return slotSharingGroup;
}

@Override
public long getDefaultBufferTimeout() {
return streamGraphGenerator.defaultBufferTimeout;
}
}
}
Loading

0 comments on commit 0c3dfc8

Please sign in to comment.