|
18 | 18 |
|
19 | 19 | package org.apache.flink.streaming.api.graph;
|
20 | 20 |
|
| 21 | +import org.apache.flink.api.common.RuntimeExecutionMode; |
21 | 22 | import org.apache.flink.api.common.eventtime.WatermarkStrategy;
|
22 | 23 | import org.apache.flink.api.common.functions.MapFunction;
|
23 | 24 | import org.apache.flink.api.common.typeinfo.Types;
|
|
32 | 33 | import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
|
33 | 34 | import org.apache.flink.streaming.api.operators.ChainingStrategy;
|
34 | 35 | import org.apache.flink.streaming.api.transformations.MultipleInputTransformation;
|
| 36 | +import org.apache.flink.streaming.api.transformations.PartitionTransformation; |
| 37 | +import org.apache.flink.streaming.api.transformations.StreamExchangeMode; |
| 38 | +import org.apache.flink.streaming.runtime.partitioner.ForwardForConsecutiveHashPartitioner; |
| 39 | +import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; |
35 | 40 |
|
36 | 41 | import org.apache.flink.shaded.guava32.com.google.common.collect.Iterables;
|
37 | 42 |
|
@@ -235,6 +240,33 @@ void testSourceChain() {
|
235 | 240 | assertThat(jobGraph.getVerticesSortedTopologicallyFromSources().size()).isEqualTo(4);
|
236 | 241 | }
|
237 | 242 |
|
| 243 | + @Test |
| 244 | + void testForwardForConsecutiveHashPartitionerChain() { |
| 245 | + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); |
| 246 | + env.setRuntimeMode(RuntimeExecutionMode.BATCH); |
| 247 | + env.setParallelism(1); |
| 248 | + |
| 249 | + final DataStream<Integer> source = env.fromData(1, 2, 3); |
| 250 | + final DataStream<Integer> forward = |
| 251 | + new DataStream<>( |
| 252 | + env, |
| 253 | + new PartitionTransformation<>( |
| 254 | + source.getTransformation(), |
| 255 | + new ForwardForConsecutiveHashPartitioner<>( |
| 256 | + new RebalancePartitioner<>()), |
| 257 | + StreamExchangeMode.BATCH)); |
| 258 | + forward.print(); |
| 259 | + |
| 260 | + StreamGraph streamGraph1 = env.getStreamGraph(false); |
| 261 | + streamGraph1.setDynamic(true); |
| 262 | + JobGraph jobGraph1 = generateJobGraphInLazilyMode(streamGraph1); |
| 263 | + |
| 264 | + StreamGraph streamGraph2 = env.getStreamGraph(false); |
| 265 | + streamGraph2.setDynamic(true); |
| 266 | + JobGraph jobGraph2 = StreamingJobGraphGenerator.createJobGraph(streamGraph2); |
| 267 | + assertThat(isJobGraphEquivalent(jobGraph1, jobGraph2)).isTrue(); |
| 268 | + } |
| 269 | + |
238 | 270 | private static JobGraph generateJobGraphInLazilyMode(StreamGraph streamGraph) {
|
239 | 271 | AdaptiveGraphManager adaptiveGraphManager =
|
240 | 272 | new AdaptiveGraphManager(
|
|
0 commit comments