Skip to content

Commit b8fce8e

Browse files
codenohupreswqa
authored andcommitted
[FLINK-37264][datastream] Fix bugs in JoinExtension for ds v2
1 parent bfea5b5 commit b8fce8e

File tree

4 files changed

+136
-22
lines changed

4 files changed

+136
-22
lines changed

flink-datastream/src/main/java/org/apache/flink/datastream/impl/stream/KeyedPartitionStreamImpl.java

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -483,15 +483,11 @@ public <T_OTHER, OUT> Transformation<OUT> getJoinTransformation(
483483
TwoInputNonBroadcastStreamProcessFunction<V, T_OTHER, OUT> processFunction,
484484
TypeInformation<OUT> outTypeInfo) {
485485
ListStateDescriptor<V> leftStateDesc =
486-
new ListStateDescriptor<>(
487-
"join-left-state", TypeExtractor.createTypeInfo(getType().getTypeClass()));
486+
new ListStateDescriptor<>("join-left-state", getType());
488487
ListStateDescriptor<T_OTHER> rightStateDesc =
489488
new ListStateDescriptor<>(
490489
"join-right-state",
491-
TypeExtractor.createTypeInfo(
492-
((KeyedPartitionStreamImpl<Object, T_OTHER>) other)
493-
.getType()
494-
.getTypeClass()));
490+
((KeyedPartitionStreamImpl<Object, T_OTHER>) other).getType());
495491
TwoInputNonBroadcastJoinProcessOperator<K, V, T_OTHER, OUT> joinProcessOperator =
496492
new TwoInputNonBroadcastJoinProcessOperator<>(
497493
processFunction, leftStateDesc, rightStateDesc);

flink-datastream/src/main/java/org/apache/flink/datastream/impl/utils/StreamUtils.java

Lines changed: 4 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -441,9 +441,7 @@ public static <K, IN, OUT, W extends Window> Transformation<OUT> transformOneInp
441441
KeySelector<IN, K> keySelector,
442442
TypeInformation<K> keyType) {
443443
WindowAssigner<IN, W> assigner = internalWindowFunction.getAssigner();
444-
ListStateDescriptor<IN> stateDesc =
445-
new ListStateDescriptor<>(
446-
"window-state", TypeExtractor.createTypeInfo(inputType.getTypeClass()));
444+
ListStateDescriptor<IN> stateDesc = new ListStateDescriptor<>("window-state", inputType);
447445

448446
OneInputWindowProcessOperator<K, IN, OUT, W> windowProcessOperator =
449447
new OneInputWindowProcessOperator<>(
@@ -474,14 +472,10 @@ Transformation<OUT> transformTwoInputNonBroadcastWindow(
474472
TypeInformation<K> keyType2) {
475473
WindowAssigner<TaggedUnion<IN1, IN2>, W> assigner = internalWindowFunction.getAssigner();
476474
ListStateDescriptor<IN1> leftStateDesc =
477-
new ListStateDescriptor<>(
478-
"two-input-window-left-state",
479-
TypeExtractor.createTypeInfo(inputType1.getTypeClass()));
475+
new ListStateDescriptor<>("two-input-window-left-state", inputType1);
480476

481477
ListStateDescriptor<IN2> rightStateDesc =
482-
new ListStateDescriptor<>(
483-
"two-input-window-right-state",
484-
TypeExtractor.createTypeInfo(inputType2.getTypeClass()));
478+
new ListStateDescriptor<>("two-input-window-right-state", inputType2);
485479

486480
TwoInputNonBroadcastWindowProcessOperator<K, IN1, IN2, OUT, W> windowProcessOperator =
487481
new TwoInputNonBroadcastWindowProcessOperator<>(
@@ -519,9 +513,7 @@ Transformation<OUT1> transformTwoOutputWindow(
519513
TypeInformation<K> keyType) {
520514
WindowAssigner<IN, W> assigner = internalWindowFunction.getAssigner();
521515
ListStateDescriptor<IN> stateDesc =
522-
new ListStateDescriptor<>(
523-
"two-output-window-state",
524-
TypeExtractor.createTypeInfo(inputType.getTypeClass()));
516+
new ListStateDescriptor<>("two-output-window-state", inputType);
525517

526518
TwoOutputWindowProcessOperator<K, IN, OUT1, OUT2, W> windowProcessOperator =
527519
new TwoOutputWindowProcessOperator<>(

flink-datastream/src/main/java/org/apache/flink/streaming/runtime/translators/DataStreamV2SinkTransformationTranslator.java

Lines changed: 47 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,8 @@
3636
import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology;
3737
import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology;
3838
import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes;
39+
import org.apache.flink.streaming.api.datastream.DataStream;
40+
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
3941
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
4042
import org.apache.flink.streaming.api.graph.TransformationTranslator;
4143
import org.apache.flink.streaming.api.transformations.DataStreamV2SinkTransformation;
@@ -140,6 +142,40 @@ public SinkExpander(
140142
this.isBatchMode = isBatchMode;
141143
}
142144

145+
/**
146+
* Checks if {@link Sink} will add commit topology.
147+
*
148+
* @param sink the sink to check
149+
* @return true if the {@link Sink} do not add a commit topology, false otherwise
150+
*/
151+
private <CommT, WriteResultT> boolean checkSinkDoNotAddCommitTopology(Sink<T> sink) {
152+
SupportsCommitter<CommT> committingSink = (SupportsCommitter<CommT>) sink;
153+
SupportsPreCommitTopology<WriteResultT, CommT> preCommittingSink =
154+
(SupportsPreCommitTopology<WriteResultT, CommT>) sink;
155+
TypeInformation<CommittableMessage<WriteResultT>> writeResultTypeInformation =
156+
CommittableMessageTypeInfo.of(preCommittingSink::getWriteResultSerializer);
157+
Transformation<CommittableMessage<WriteResultT>> dummyTransformation =
158+
new Transformation<>("dummy", writeResultTypeInformation, 1) {
159+
@Override
160+
protected List<Transformation<?>> getTransitivePredecessorsInternal() {
161+
return List.of();
162+
}
163+
164+
@Override
165+
public List<Transformation<?>> getInputs() {
166+
return List.of();
167+
}
168+
};
169+
DataStream<CommittableMessage<CommT>> committableMessageDataStream =
170+
preCommittingSink.addPreCommitTopology(
171+
new DataStream<>(
172+
new StreamExecutionEnvironment(), dummyTransformation));
173+
return committableMessageDataStream
174+
.getExecutionEnvironment()
175+
.getTransformations()
176+
.isEmpty();
177+
}
178+
143179
private void expand() {
144180

145181
final int sizeBefore = executionEnvironment.getTransformations().size();
@@ -150,8 +186,17 @@ private void expand() {
150186
throw new UnsupportedOperationException(
151187
"Sink with pre-write topology is not supported for DataStream v2 atm.");
152188
} else if (sink instanceof SupportsPreCommitTopology) {
153-
throw new UnsupportedOperationException(
154-
"Sink with pre-commit topology is not supported for DataStream v2 atm.");
189+
if (sink.getClass()
190+
.getName()
191+
.equals("org.apache.flink.connector.file.sink.FileSink")) {
192+
if (!checkSinkDoNotAddCommitTopology(sink)) {
193+
throw new UnsupportedOperationException(
194+
"Sink with pre-commit topology is not supported for DataStream v2 atm.");
195+
}
196+
} else {
197+
throw new UnsupportedOperationException(
198+
"Sink with pre-commit topology is not supported for DataStream v2 atm.");
199+
}
155200
} else if (sink instanceof SupportsPostCommitTopology) {
156201
throw new UnsupportedOperationException(
157202
"Sink with post-commit topology is not supported for DataStream v2 atm.");

flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/JoinITCase.java renamed to flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/extension/join/JoinITCase.java

Lines changed: 83 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,14 +16,20 @@
1616
* limitations under the License.
1717
*/
1818

19-
package org.apache.flink.test.streaming.api.datastream;
19+
package org.apache.flink.test.streaming.api.datastream.extension.join;
2020

21+
import org.apache.flink.api.common.serialization.SimpleStringEncoder;
2122
import org.apache.flink.api.connector.dsv2.DataStreamV2SourceUtils;
2223
import org.apache.flink.api.connector.dsv2.WrappedSink;
2324
import org.apache.flink.api.connector.sink2.Sink;
2425
import org.apache.flink.api.connector.sink2.SinkWriter;
2526
import org.apache.flink.api.connector.sink2.WriterInitContext;
2627
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;
2733
import org.apache.flink.datastream.api.ExecutionEnvironment;
2834
import org.apache.flink.datastream.api.builtin.BuiltinFuncs;
2935
import org.apache.flink.datastream.api.common.Collector;
@@ -35,23 +41,27 @@
3541
import org.apache.flink.datastream.api.function.TwoInputNonBroadcastStreamProcessFunction;
3642
import org.apache.flink.datastream.api.stream.KeyedPartitionStream;
3743
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;
3846

3947
import org.junit.jupiter.api.AfterEach;
4048
import org.junit.jupiter.api.BeforeEach;
4149
import org.junit.jupiter.api.Test;
4250

4351
import java.io.IOException;
4452
import java.io.Serializable;
53+
import java.time.Duration;
4554
import java.util.ArrayList;
4655
import java.util.Arrays;
4756
import java.util.Collection;
4857
import java.util.Collections;
4958
import java.util.List;
5059

60+
import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory;
5161
import static org.assertj.core.api.Assertions.assertThat;
5262

5363
/** Test the Join extension on DataStream V2. */
54-
class JoinITCase implements Serializable {
64+
class JoinITCase extends AbstractTestBase implements Serializable {
5565
private transient ExecutionEnvironment env;
5666
private static List<String> sinkResults;
5767

@@ -299,6 +309,77 @@ void testJoinWithNonKeyedStream() throws Exception {
299309
"key:2:1", "key:2:2");
300310
}
301311

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+
302383
private static class KeyAndValue {
303384
public final String key;
304385
public final int value;

0 commit comments

Comments
 (0)