Skip to content

Commit b9b41bc

Browse files
[FLINK-32695] Moved TestSource in TypeFillTest to SourceV2
1 parent 5753b09 commit b9b41bc

File tree

2 files changed

+19
-12
lines changed

2 files changed

+19
-12
lines changed

flink-runtime/pom.xml

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -92,6 +92,13 @@ under the License.
9292
<scope>test</scope>
9393
</dependency>
9494

95+
<dependency>
96+
<groupId>org.apache.flink</groupId>
97+
<artifactId>flink-test-utils-connector</artifactId>
98+
<version>${project.version}</version>
99+
<scope>test</scope>
100+
</dependency>
101+
95102
<dependency>
96103
<groupId>org.apache.flink</groupId>
97104
<artifactId>flink-queryable-state-client-java</artifactId>

flink-runtime/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java

Lines changed: 12 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.flink.streaming.api;
1919

20+
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
2021
import org.apache.flink.api.common.functions.FlatMapFunction;
2122
import org.apache.flink.api.common.functions.InvalidTypesException;
2223
import org.apache.flink.api.common.functions.MapFunction;
@@ -30,7 +31,7 @@
3031
import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
3132
import org.apache.flink.streaming.api.functions.co.CoMapFunction;
3233
import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction;
33-
import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction;
34+
import org.apache.flink.test.util.source.AbstractTestSource;
3435
import org.apache.flink.util.Collector;
3536

3637
import org.junit.jupiter.api.Test;
@@ -51,7 +52,13 @@ class TypeFillTest {
5152
void test() {
5253
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
5354

54-
assertThatThrownBy(() -> env.addSource(new TestSource<Integer>()).print())
55+
assertThatThrownBy(
56+
() ->
57+
env.fromSource(
58+
new TestSource<Integer>(),
59+
WatermarkStrategy.noWatermarks(),
60+
"")
61+
.print())
5562
.isInstanceOf(InvalidTypesException.class);
5663

5764
DataStream<Long> source = env.fromSequence(1, 10);
@@ -110,7 +117,8 @@ void test() {
110117
.print())
111118
.isInstanceOf(InvalidTypesException.class);
112119

113-
env.addSource(new TestSource<Integer>()).returns(Integer.class);
120+
env.fromSource(new TestSource<Integer>(), WatermarkStrategy.noWatermarks(), "")
121+
.returns(Integer.class);
114122
source.map(new TestMap<Long, Long>()).returns(Long.class).print();
115123
source.flatMap(new TestFlatMap<Long, Long>()).returns(new TypeHint<Long>() {}).print();
116124
source.connect(source)
@@ -157,15 +165,7 @@ public String map(Long value) throws Exception {
157165
.isInstanceOf(IllegalStateException.class);
158166
}
159167

160-
private static class TestSource<T> implements SourceFunction<T> {
161-
private static final long serialVersionUID = 1L;
162-
163-
@Override
164-
public void run(SourceContext<T> ctx) throws Exception {}
165-
166-
@Override
167-
public void cancel() {}
168-
}
168+
private static class TestSource<T> extends AbstractTestSource<T> {}
169169

170170
private static class TestMap<T, O> implements MapFunction<T, O> {
171171
@Override

0 commit comments

Comments
 (0)