|
16 | 16 | * limitations under the License.
|
17 | 17 | */
|
18 | 18 |
|
19 |
| -package org.apache.flink.test.streaming.api.datastream; |
| 19 | +package org.apache.flink.test.streaming.api.datastream.extension.join; |
20 | 20 |
|
| 21 | +import org.apache.flink.api.common.serialization.SimpleStringEncoder; |
21 | 22 | import org.apache.flink.api.connector.dsv2.DataStreamV2SourceUtils;
|
22 | 23 | import org.apache.flink.api.connector.dsv2.WrappedSink;
|
23 | 24 | import org.apache.flink.api.connector.sink2.Sink;
|
24 | 25 | import org.apache.flink.api.connector.sink2.SinkWriter;
|
25 | 26 | import org.apache.flink.api.connector.sink2.WriterInitContext;
|
26 | 27 | import org.apache.flink.api.java.functions.KeySelector;
|
| 28 | +import org.apache.flink.api.java.tuple.Tuple2; |
| 29 | +import org.apache.flink.api.java.tuple.Tuple3; |
| 30 | +import org.apache.flink.configuration.MemorySize; |
| 31 | +import org.apache.flink.connector.file.sink.FileSink; |
| 32 | +import org.apache.flink.core.fs.Path; |
27 | 33 | import org.apache.flink.datastream.api.ExecutionEnvironment;
|
28 | 34 | import org.apache.flink.datastream.api.builtin.BuiltinFuncs;
|
29 | 35 | import org.apache.flink.datastream.api.common.Collector;
|
|
35 | 41 | import org.apache.flink.datastream.api.function.TwoInputNonBroadcastStreamProcessFunction;
|
36 | 42 | import org.apache.flink.datastream.api.stream.KeyedPartitionStream;
|
37 | 43 | import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream;
|
| 44 | +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; |
| 45 | +import org.apache.flink.test.util.AbstractTestBase; |
38 | 46 |
|
39 | 47 | import org.junit.jupiter.api.AfterEach;
|
40 | 48 | import org.junit.jupiter.api.BeforeEach;
|
41 | 49 | import org.junit.jupiter.api.Test;
|
42 | 50 |
|
43 | 51 | import java.io.IOException;
|
44 | 52 | import java.io.Serializable;
|
| 53 | +import java.time.Duration; |
45 | 54 | import java.util.ArrayList;
|
46 | 55 | import java.util.Arrays;
|
47 | 56 | import java.util.Collection;
|
48 | 57 | import java.util.Collections;
|
49 | 58 | import java.util.List;
|
50 | 59 |
|
| 60 | +import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; |
51 | 61 | import static org.assertj.core.api.Assertions.assertThat;
|
52 | 62 |
|
53 | 63 | /** Test the Join extension on DataStream V2. */
|
54 |
| -class JoinITCase implements Serializable { |
| 64 | +class JoinITCase extends AbstractTestBase implements Serializable { |
55 | 65 | private transient ExecutionEnvironment env;
|
56 | 66 | private static List<String> sinkResults;
|
57 | 67 |
|
@@ -299,6 +309,77 @@ void testJoinWithNonKeyedStream() throws Exception {
|
299 | 309 | "key:2:1", "key:2:2");
|
300 | 310 | }
|
301 | 311 |
|
| 312 | + @Test |
| 313 | + void testJoinWithTuple() throws Exception { |
| 314 | + final String resultPath = getTempDirPath("result"); |
| 315 | + |
| 316 | + NonKeyedPartitionStream<Tuple2<String, Integer>> source1 = |
| 317 | + env.fromSource( |
| 318 | + DataStreamV2SourceUtils.fromData( |
| 319 | + Arrays.asList( |
| 320 | + Tuple2.of("key", 0), |
| 321 | + Tuple2.of("key", 1), |
| 322 | + Tuple2.of("key", 2))), |
| 323 | + "source1"); |
| 324 | + |
| 325 | + NonKeyedPartitionStream<Tuple2<String, Integer>> source2 = |
| 326 | + env.fromSource( |
| 327 | + DataStreamV2SourceUtils.fromData( |
| 328 | + Arrays.asList( |
| 329 | + Tuple2.of("key", 0), |
| 330 | + Tuple2.of("key", 1), |
| 331 | + Tuple2.of("key", 2))), |
| 332 | + "source2"); |
| 333 | + |
| 334 | + NonKeyedPartitionStream<Tuple3<String, Integer, Integer>> joinedStream = |
| 335 | + BuiltinFuncs.join( |
| 336 | + source1, |
| 337 | + (KeySelector<Tuple2<String, Integer>, String>) elem -> elem.f0, |
| 338 | + source2, |
| 339 | + (KeySelector<Tuple2<String, Integer>, String>) elem -> elem.f0, |
| 340 | + new JoinFunction< |
| 341 | + Tuple2<String, Integer>, |
| 342 | + Tuple2<String, Integer>, |
| 343 | + Tuple3<String, Integer, Integer>>() { |
| 344 | + |
| 345 | + @Override |
| 346 | + public void processRecord( |
| 347 | + Tuple2<String, Integer> leftRecord, |
| 348 | + Tuple2<String, Integer> rightRecord, |
| 349 | + Collector<Tuple3<String, Integer, Integer>> output, |
| 350 | + RuntimeContext ctx) |
| 351 | + throws Exception { |
| 352 | + output.collect( |
| 353 | + Tuple3.of(leftRecord.f0, leftRecord.f1, rightRecord.f1)); |
| 354 | + } |
| 355 | + }); |
| 356 | + |
| 357 | + joinedStream.toSink( |
| 358 | + new WrappedSink<>( |
| 359 | + FileSink.<Tuple3<String, Integer, Integer>>forRowFormat( |
| 360 | + new Path(resultPath), new SimpleStringEncoder<>()) |
| 361 | + .withRollingPolicy( |
| 362 | + DefaultRollingPolicy.builder() |
| 363 | + .withMaxPartSize(MemorySize.ofMebiBytes(1)) |
| 364 | + .withRolloverInterval(Duration.ofSeconds(10)) |
| 365 | + .build()) |
| 366 | + .build())); |
| 367 | + |
| 368 | + env.execute("testJoinWithTuple"); |
| 369 | + |
| 370 | + compareResultsByLinesInMemory( |
| 371 | + "(key,0,0)\n" |
| 372 | + + "(key,0,1)\n" |
| 373 | + + "(key,0,2)\n" |
| 374 | + + "(key,1,0)\n" |
| 375 | + + "(key,1,1)\n" |
| 376 | + + "(key,1,2)\n" |
| 377 | + + "(key,2,0)\n" |
| 378 | + + "(key,2,1)\n" |
| 379 | + + "(key,2,2)\n", |
| 380 | + resultPath); |
| 381 | + } |
| 382 | + |
302 | 383 | private static class KeyAndValue {
|
303 | 384 | public final String key;
|
304 | 385 | public final int value;
|
|
0 commit comments