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-20491] Turn BroadcastStateTransformation into "logical" Transf…
…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
Showing
7 changed files
with
342 additions
and
171 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
107 changes: 107 additions & 0 deletions
107
.../org/apache/flink/streaming/api/transformations/AbstractBroadcastStateTransformation.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,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; | ||
} | ||
} |
Oops, something went wrong.