[FLINK-20491] Turn BroadcastStateTransformation into "logical" Transformation

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.
上级 0a66d09b
......@@ -27,11 +27,8 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
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;
......@@ -119,13 +116,13 @@ public class BroadcastConnectedStream<IN1, IN2> {
*
* @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(
......@@ -150,13 +147,13 @@ public class BroadcastConnectedStream<IN1, IN2> {
* @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);
......@@ -164,9 +161,7 @@ public class BroadcastConnectedStream<IN1, IN2> {
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);
}
/**
......@@ -220,23 +215,27 @@ public class BroadcastConnectedStream<IN1, IN2> {
!(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 =
......@@ -246,28 +245,35 @@ public class BroadcastConnectedStream<IN1, IN2> {
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) {
......
......@@ -39,6 +39,7 @@ import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionState
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;
......@@ -55,6 +56,7 @@ import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
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;
......@@ -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);
}
......
/*
* 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;
}
}
......@@ -19,146 +19,40 @@
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.api.java.functions.KeySelector;
import org.apache.flink.streaming.api.datastream.BroadcastStream;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.KeyedStream;
import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* This is the transformation for the Broadcast State pattern. In a nutshell, this transformation
* allows to take a broadcasted (non-keyed) stream, connect it with another keyed or non-keyed
* stream, and apply a function on the resulting connected stream. This function will have access to
* all the elements that belong to the non-keyed, broadcasted side, as this is kept in Flink's
* state.
*
* <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.
*/
/** A non-keyed {@link AbstractBroadcastStateTransformation}. */
@Internal
public class BroadcastStateTransformation<IN1, IN2, OUT> extends PhysicalTransformation<OUT> {
public class BroadcastStateTransformation<IN1, IN2, OUT>
extends AbstractBroadcastStateTransformation<IN1, IN2, OUT> {
private final Transformation<IN1> nonBroadcastStream;
private final BroadcastProcessFunction<IN1, IN2, OUT> userFunction;
private final Transformation<IN2> broadcastStream;
private final StreamOperatorFactory<OUT> operatorFactory;
private final TypeInformation<?> stateKeyType;
private final KeySelector<IN1, ?> keySelector;
private BroadcastStateTransformation(
public BroadcastStateTransformation(
final String name,
final Transformation<IN1> inputStream,
final Transformation<IN2> broadcastStream,
final StreamOperatorFactory<OUT> operatorFactory,
@Nullable final TypeInformation<?> keyType,
@Nullable final KeySelector<IN1, ?> keySelector,
final TypeInformation<OUT> outTypeInfo,
final int parallelism) {
super(name, outTypeInfo, parallelism);
this.nonBroadcastStream = checkNotNull(inputStream);
this.broadcastStream = checkNotNull(broadcastStream);
this.operatorFactory = checkNotNull(operatorFactory);
this.stateKeyType = keyType;
this.keySelector = keySelector;
updateManagedMemoryStateBackendUseCase(keySelector != null);
}
public Transformation<IN2> getBroadcastStream() {
return broadcastStream;
}
public Transformation<IN1> getNonBroadcastStream() {
return nonBroadcastStream;
}
public StreamOperatorFactory<OUT> getOperatorFactory() {
return operatorFactory;
}
public TypeInformation<?> getStateKeyType() {
return stateKeyType;
}
public KeySelector<IN1, ?> getKeySelector() {
return keySelector;
}
@Override
public void setChainingStrategy(ChainingStrategy strategy) {
this.operatorFactory.getChainingStrategy();
}
@Override
public List<Transformation<?>> getTransitivePredecessors() {
final List<Transformation<?>> predecessors = new ArrayList<>();
predecessors.add(this);
predecessors.add(nonBroadcastStream);
predecessors.add(broadcastStream);
return predecessors;
}
@Override
public List<Transformation<?>> getInputs() {
final List<Transformation<?>> predecessors = new ArrayList<>();
predecessors.add(nonBroadcastStream);
predecessors.add(broadcastStream);
return predecessors;
}
// ------------------------------- Static Constructors -------------------------------
public static <IN1, IN2, OUT> BroadcastStateTransformation<IN1, IN2, OUT> forNonKeyedStream(
final String name,
final DataStream<IN1> nonBroadcastStream,
final BroadcastStream<IN2> broadcastStream,
final StreamOperatorFactory<OUT> operatorFactory,
final BroadcastProcessFunction<IN1, IN2, OUT> userFunction,
final List<MapStateDescriptor<?, ?>> broadcastStateDescriptors,
final TypeInformation<OUT> outTypeInfo,
final int parallelism) {
return new BroadcastStateTransformation<>(
super(
name,
checkNotNull(nonBroadcastStream).getTransformation(),
checkNotNull(broadcastStream).getTransformation(),
operatorFactory,
null,
null,
inputStream,
broadcastStream,
broadcastStateDescriptors,
outTypeInfo,
parallelism);
this.userFunction = userFunction;
updateManagedMemoryStateBackendUseCase(false /* not keyed */);
}
public static <IN1, IN2, OUT> BroadcastStateTransformation<IN1, IN2, OUT> forKeyedStream(
final String name,
final KeyedStream<IN1, ?> nonBroadcastStream,
final BroadcastStream<IN2> broadcastStream,
final StreamOperatorFactory<OUT> operatorFactory,
final TypeInformation<OUT> outTypeInfo,
final int parallelism) {
return new BroadcastStateTransformation<>(
name,
checkNotNull(nonBroadcastStream).getTransformation(),
checkNotNull(broadcastStream).getTransformation(),
operatorFactory,
nonBroadcastStream.getKeyType(),
nonBroadcastStream.getKeySelector(),
outTypeInfo,
parallelism);
public BroadcastProcessFunction<IN1, IN2, OUT> getUserFunction() {
return userFunction;
}
}
/*
* 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.api.java.functions.KeySelector;
import org.apache.flink.streaming.api.functions.co.KeyedBroadcastProcessFunction;
import java.util.List;
/** A keyed {@link AbstractBroadcastStateTransformation}. */
@Internal
public class KeyedBroadcastStateTransformation<KEY, IN1, IN2, OUT>
extends AbstractBroadcastStateTransformation<IN1, IN2, OUT> {
private final KeyedBroadcastProcessFunction<KEY, IN1, IN2, OUT> userFunction;
private final TypeInformation<KEY> stateKeyType;
private final KeySelector<IN1, KEY> keySelector;
public KeyedBroadcastStateTransformation(
final String name,
final Transformation<IN1> inputStream,
final Transformation<IN2> broadcastStream,
final KeyedBroadcastProcessFunction<KEY, IN1, IN2, OUT> userFunction,
final List<MapStateDescriptor<?, ?>> broadcastStateDescriptors,
final TypeInformation<KEY> keyType,
final KeySelector<IN1, KEY> keySelector,
final TypeInformation<OUT> outTypeInfo,
final int parallelism) {
super(
name,
inputStream,
broadcastStream,
broadcastStateDescriptors,
outTypeInfo,
parallelism);
this.userFunction = userFunction;
this.stateKeyType = keyType;
this.keySelector = keySelector;
updateManagedMemoryStateBackendUseCase(true /* we have keyed state */);
}
public KeyedBroadcastProcessFunction<KEY, IN1, IN2, OUT> getUserFunction() {
return userFunction;
}
public TypeInformation<KEY> getStateKeyType() {
return stateKeyType;
}
public KeySelector<IN1, KEY> getKeySelector() {
return keySelector;
}
}
......@@ -20,6 +20,8 @@ package org.apache.flink.streaming.runtime.translators;
import org.apache.flink.annotation.Internal;
import org.apache.flink.streaming.api.graph.TransformationTranslator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
import org.apache.flink.streaming.api.operators.co.CoBroadcastWithNonKeyedOperator;
import org.apache.flink.streaming.api.transformations.BroadcastStateTransformation;
import java.util.Collection;
......@@ -55,14 +57,19 @@ public class BroadcastStateTransformationTranslator<IN1, IN2, OUT>
checkNotNull(transformation);
checkNotNull(context);
CoBroadcastWithNonKeyedOperator<IN1, IN2, OUT> operator =
new CoBroadcastWithNonKeyedOperator<>(
transformation.getUserFunction(),
transformation.getBroadcastStateDescriptors());
return translateInternal(
transformation,
transformation.getNonBroadcastStream(),
transformation.getBroadcastStream(),
transformation.getOperatorFactory(),
transformation.getStateKeyType(),
transformation.getKeySelector(),
null,
transformation.getRegularInput(),
transformation.getBroadcastInput(),
SimpleOperatorFactory.of(operator),
null /* no key type*/,
null /* no first key selector */,
null /* no key selector on broadcast input*/,
context);
}
}
/*
* 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.translators;
import org.apache.flink.annotation.Internal;
import org.apache.flink.streaming.api.graph.TransformationTranslator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
import org.apache.flink.streaming.api.operators.co.CoBroadcastWithKeyedOperator;
import org.apache.flink.streaming.api.transformations.KeyedBroadcastStateTransformation;
import java.util.Collection;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* A {@link TransformationTranslator} for the {@link KeyedBroadcastStateTransformation}.
*
* @param <IN1> The type of the elements in the non-broadcasted input of the {@link
* KeyedBroadcastStateTransformation}.
* @param <IN2> The type of the elements in the broadcasted input of the {@link
* KeyedBroadcastStateTransformation}.
* @param <OUT> The type of the elements that result from the {@link
* KeyedBroadcastStateTransformation}.
*/
@Internal
public class KeyedBroadcastStateTransformationTranslator<KEY, IN1, IN2, OUT>
extends AbstractTwoInputTransformationTranslator<
IN1, IN2, OUT, KeyedBroadcastStateTransformation<KEY, IN1, IN2, OUT>> {
@Override
protected Collection<Integer> translateForBatchInternal(
final KeyedBroadcastStateTransformation<KEY, IN1, IN2, OUT> transformation,
final Context context) {
throw new UnsupportedOperationException(
"The Broadcast State Pattern is not support in BATCH execution mode.");
}
@Override
protected Collection<Integer> translateForStreamingInternal(
final KeyedBroadcastStateTransformation<KEY, IN1, IN2, OUT> transformation,
final Context context) {
checkNotNull(transformation);
checkNotNull(context);
CoBroadcastWithKeyedOperator<KEY, IN1, IN2, OUT> operator =
new CoBroadcastWithKeyedOperator<>(
transformation.getUserFunction(),
transformation.getBroadcastStateDescriptors());
return translateInternal(
transformation,
transformation.getRegularInput(),
transformation.getBroadcastInput(),
SimpleOperatorFactory.of(operator),
transformation.getStateKeyType(),
transformation.getKeySelector(),
null /* no key selector on broadcast input */,
context);
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册