Skip to content

Commit

Permalink
[FLINK-20491] Turn BroadcastStateTransformation into "logical" Transf…
Browse files Browse the repository at this point in the history
…ormation

Note: Broadcast operations in BATCH mode don't yet work with this
change. This needs follow-up changes from later commits. We just lay the
groundwork here and keep the same functionality.

Before, we were creating operators in BroadcastConnectedStreams eagerly.
Now, the transformation holds the user function and we add a Translator
that creates the "physical" operators when translating the graph of
Transformations.

We do this so that we can translate differently based on whether we're
in BATCH or STREAMING mode.
  • Loading branch information
aljoscha committed Jan 7, 2021
1 parent 0a66d09 commit e31b162
Show file tree
Hide file tree
Showing 7 changed files with 342 additions and 171 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,8 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction;
import org.apache.flink.streaming.api.functions.co.KeyedBroadcastProcessFunction;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.operators.co.CoBroadcastWithKeyedOperator;
import org.apache.flink.streaming.api.operators.co.CoBroadcastWithNonKeyedOperator;
import org.apache.flink.streaming.api.transformations.BroadcastStateTransformation;
import org.apache.flink.streaming.api.transformations.KeyedBroadcastStateTransformation;
import org.apache.flink.util.Preconditions;

import java.util.List;
Expand Down Expand Up @@ -119,13 +116,13 @@ public TypeInformation<IN2> getType2() {
*
* @param function The {@link KeyedBroadcastProcessFunction} that is called for each element in
* the stream.
* @param <KS> The type of the keys in the keyed stream.
* @param <KEY> The type of the keys in the keyed stream.
* @param <OUT> The type of the output elements.
* @return The transformed {@link DataStream}.
*/
@PublicEvolving
public <KS, OUT> SingleOutputStreamOperator<OUT> process(
final KeyedBroadcastProcessFunction<KS, IN1, IN2, OUT> function) {
public <KEY, OUT> SingleOutputStreamOperator<OUT> process(
final KeyedBroadcastProcessFunction<KEY, IN1, IN2, OUT> function) {

TypeInformation<OUT> outTypeInfo =
TypeExtractor.getBinaryOperatorReturnType(
Expand All @@ -150,23 +147,21 @@ public <KS, OUT> SingleOutputStreamOperator<OUT> process(
* @param function The {@link KeyedBroadcastProcessFunction} that is called for each element in
* the stream.
* @param outTypeInfo The type of the output elements.
* @param <KS> The type of the keys in the keyed stream.
* @param <KEY> The type of the keys in the keyed stream.
* @param <OUT> The type of the output elements.
* @return The transformed {@link DataStream}.
*/
@PublicEvolving
public <KS, OUT> SingleOutputStreamOperator<OUT> process(
final KeyedBroadcastProcessFunction<KS, IN1, IN2, OUT> function,
public <KEY, OUT> SingleOutputStreamOperator<OUT> process(
final KeyedBroadcastProcessFunction<KEY, IN1, IN2, OUT> function,
final TypeInformation<OUT> outTypeInfo) {

Preconditions.checkNotNull(function);
Preconditions.checkArgument(
nonBroadcastStream instanceof KeyedStream,
"A KeyedBroadcastProcessFunction can only be used on a keyed stream.");

TwoInputStreamOperator<IN1, IN2, OUT> operator =
new CoBroadcastWithKeyedOperator<>(clean(function), broadcastStateDescriptors);
return transform("Co-Process-Broadcast-Keyed", outTypeInfo, operator);
return transform(function, outTypeInfo);
}

/**
Expand Down Expand Up @@ -220,23 +215,27 @@ public <OUT> SingleOutputStreamOperator<OUT> process(
!(nonBroadcastStream instanceof KeyedStream),
"A BroadcastProcessFunction can only be used on a non-keyed stream.");

TwoInputStreamOperator<IN1, IN2, OUT> operator =
new CoBroadcastWithNonKeyedOperator<>(clean(function), broadcastStateDescriptors);
return transform("Co-Process-Broadcast", outTypeInfo, operator);
return transform(function, outTypeInfo);
}

@Internal
private <OUT> SingleOutputStreamOperator<OUT> transform(
final String functionName,
final TypeInformation<OUT> outTypeInfo,
final TwoInputStreamOperator<IN1, IN2, OUT> operator) {
final BroadcastProcessFunction<IN1, IN2, OUT> userFunction,
final TypeInformation<OUT> outTypeInfo) {

// read the output type of the input Transforms to coax out errors about MissingTypeInfo
nonBroadcastStream.getType();
broadcastStream.getType();

final BroadcastStateTransformation<IN1, IN2, OUT> transformation =
getBroadcastStateTransformation(functionName, outTypeInfo, operator);
new BroadcastStateTransformation<>(
"Co-Process-Broadcast",
nonBroadcastStream.getTransformation(),
broadcastStream.getTransformation(),
clean(userFunction),
broadcastStateDescriptors,
outTypeInfo,
environment.getParallelism());

@SuppressWarnings({"unchecked", "rawtypes"})
final SingleOutputStreamOperator<OUT> returnStream =
Expand All @@ -246,28 +245,35 @@ private <OUT> SingleOutputStreamOperator<OUT> transform(
return returnStream;
}

private <OUT> BroadcastStateTransformation<IN1, IN2, OUT> getBroadcastStateTransformation(
final String functionName,
final TypeInformation<OUT> outTypeInfo,
final TwoInputStreamOperator<IN1, IN2, OUT> operator) {

if (nonBroadcastStream instanceof KeyedStream) {
return BroadcastStateTransformation.forKeyedStream(
functionName,
(KeyedStream<IN1, ?>) nonBroadcastStream,
broadcastStream,
SimpleOperatorFactory.of(operator),
outTypeInfo,
environment.getParallelism());
} else {
return BroadcastStateTransformation.forNonKeyedStream(
functionName,
nonBroadcastStream,
broadcastStream,
SimpleOperatorFactory.of(operator),
outTypeInfo,
environment.getParallelism());
}
@Internal
private <KEY, OUT> SingleOutputStreamOperator<OUT> transform(
final KeyedBroadcastProcessFunction<KEY, IN1, IN2, OUT> userFunction,
final TypeInformation<OUT> outTypeInfo) {

// read the output type of the input Transforms to coax out errors about MissingTypeInfo
nonBroadcastStream.getType();
broadcastStream.getType();

KeyedStream<IN1, KEY> keyedInputStream = (KeyedStream<IN1, KEY>) nonBroadcastStream;

final KeyedBroadcastStateTransformation<KEY, IN1, IN2, OUT> transformation =
new KeyedBroadcastStateTransformation<>(
"Co-Process-Broadcast-Keyed",
nonBroadcastStream.getTransformation(),
broadcastStream.getTransformation(),
clean(userFunction),
broadcastStateDescriptors,
keyedInputStream.getKeyType(),
keyedInputStream.getKeySelector(),
outTypeInfo,
environment.getParallelism());

@SuppressWarnings({"unchecked", "rawtypes"})
final SingleOutputStreamOperator<OUT> returnStream =
new SingleOutputStreamOperator(environment, transformation);

getExecutionEnvironment().addOperator(transformation);
return returnStream;
}

protected <F> F clean(F f) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.flink.streaming.api.transformations.BroadcastStateTransformation;
import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
import org.apache.flink.streaming.api.transformations.KeyedBroadcastStateTransformation;
import org.apache.flink.streaming.api.transformations.KeyedMultipleInputTransformation;
import org.apache.flink.streaming.api.transformations.LegacySinkTransformation;
import org.apache.flink.streaming.api.transformations.LegacySourceTransformation;
Expand All @@ -55,6 +56,7 @@
import org.apache.flink.streaming.api.transformations.UnionTransformation;
import org.apache.flink.streaming.api.transformations.WithBoundedness;
import org.apache.flink.streaming.runtime.translators.BroadcastStateTransformationTranslator;
import org.apache.flink.streaming.runtime.translators.KeyedBroadcastStateTransformationTranslator;
import org.apache.flink.streaming.runtime.translators.LegacySinkTransformationTranslator;
import org.apache.flink.streaming.runtime.translators.LegacySourceTransformationTranslator;
import org.apache.flink.streaming.runtime.translators.MultiInputTransformationTranslator;
Expand Down Expand Up @@ -177,6 +179,9 @@ public class StreamGraphGenerator {
TimestampsAndWatermarksTransformation.class,
new TimestampsAndWatermarksTransformationTranslator<>());
tmp.put(BroadcastStateTransformation.class, new BroadcastStateTransformationTranslator<>());
tmp.put(
KeyedBroadcastStateTransformation.class,
new KeyedBroadcastStateTransformationTranslator<>());
translatorMap = Collections.unmodifiableMap(tmp);
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
/*
* 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.transformations;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.MapStateDescriptor;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.ChainingStrategy;

import java.util.ArrayList;
import java.util.List;

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

/**
* Base class for Broadcast State transformations. In a nutshell, this transformation allows to take
* a broadcast (non-keyed) stream, connect it with another keyed or non-keyed stream, and apply a
* function on the resulting connected stream.
*
* <p>For more information see the <a
* href="https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/broadcast_state.html">
* Broadcast State Pattern documentation page</a>.
*
* @param <IN1> The type of the elements in the non-broadcasted input.
* @param <IN2> The type of the elements in the broadcasted input.
* @param <OUT> The type of the elements that result from this transformation.
*/
@Internal
public class AbstractBroadcastStateTransformation<IN1, IN2, OUT>
extends PhysicalTransformation<OUT> {

private final List<MapStateDescriptor<?, ?>> broadcastStateDescriptors;

private final Transformation<IN1> regularInput;

private final Transformation<IN2> broadcastInput;

private ChainingStrategy chainingStrategy = ChainingStrategy.DEFAULT_CHAINING_STRATEGY;

protected AbstractBroadcastStateTransformation(
final String name,
final Transformation<IN1> regularInput,
final Transformation<IN2> broadcastInput,
final List<MapStateDescriptor<?, ?>> broadcastStateDescriptors,
final TypeInformation<OUT> outTypeInfo,
final int parallelism) {
super(name, outTypeInfo, parallelism);
this.regularInput = checkNotNull(regularInput);
this.broadcastInput = checkNotNull(broadcastInput);
this.broadcastStateDescriptors = broadcastStateDescriptors;
}

public Transformation<IN2> getBroadcastInput() {
return broadcastInput;
}

public Transformation<IN1> getRegularInput() {
return regularInput;
}

public List<MapStateDescriptor<?, ?>> getBroadcastStateDescriptors() {
return broadcastStateDescriptors;
}

public ChainingStrategy getChainingStrategy() {
return chainingStrategy;
}

@Override
public void setChainingStrategy(ChainingStrategy chainingStrategy) {
this.chainingStrategy = checkNotNull(chainingStrategy);
}

@Override
public List<Transformation<?>> getTransitivePredecessors() {
final List<Transformation<?>> predecessors = new ArrayList<>();
predecessors.add(this);
predecessors.add(regularInput);
predecessors.add(broadcastInput);
return predecessors;
}

@Override
public List<Transformation<?>> getInputs() {
final List<Transformation<?>> predecessors = new ArrayList<>();
predecessors.add(regularInput);
predecessors.add(broadcastInput);
return predecessors;
}
}
Loading

0 comments on commit e31b162

Please sign in to comment.