20
20
21
21
import org .apache .flink .api .common .ExecutionConfig ;
22
22
import org .apache .flink .api .common .RuntimeExecutionMode ;
23
+ import org .apache .flink .api .common .eventtime .WatermarkStrategy ;
24
+ import org .apache .flink .api .common .functions .InvalidTypesException ;
25
+ import org .apache .flink .api .common .typeinfo .TypeInformation ;
26
+ import org .apache .flink .api .connector .dsv2 .FromDataSource ;
27
+ import org .apache .flink .api .connector .dsv2 .Source ;
28
+ import org .apache .flink .api .connector .dsv2 .WrappedSource ;
23
29
import org .apache .flink .api .dag .Transformation ;
30
+ import org .apache .flink .api .java .typeutils .MissingTypeInfo ;
31
+ import org .apache .flink .api .java .typeutils .ResultTypeQueryable ;
32
+ import org .apache .flink .api .java .typeutils .TypeExtractor ;
24
33
import org .apache .flink .configuration .Configuration ;
25
34
import org .apache .flink .configuration .DeploymentOptions ;
26
35
import org .apache .flink .configuration .ExecutionOptions ;
27
36
import org .apache .flink .configuration .ReadableConfig ;
37
+ import org .apache .flink .connector .datagen .functions .FromElementsGeneratorFunction ;
38
+ import org .apache .flink .connector .datagen .source .DataGeneratorSource ;
28
39
import org .apache .flink .core .execution .DefaultExecutorServiceLoader ;
29
40
import org .apache .flink .core .execution .JobClient ;
30
41
import org .apache .flink .core .execution .PipelineExecutor ;
31
42
import org .apache .flink .core .execution .PipelineExecutorFactory ;
32
43
import org .apache .flink .core .execution .PipelineExecutorServiceLoader ;
33
44
import org .apache .flink .datastream .api .ExecutionEnvironment ;
45
+ import org .apache .flink .datastream .api .stream .NonKeyedPartitionStream ;
46
+ import org .apache .flink .datastream .impl .stream .NonKeyedPartitionStreamImpl ;
34
47
import org .apache .flink .streaming .api .environment .CheckpointConfig ;
35
48
import org .apache .flink .streaming .api .graph .StreamGraph ;
36
49
import org .apache .flink .streaming .api .graph .StreamGraphGenerator ;
50
+ import org .apache .flink .streaming .api .transformations .SourceTransformation ;
37
51
import org .apache .flink .util .ExceptionUtils ;
38
52
import org .apache .flink .util .FlinkException ;
53
+ import org .apache .flink .util .Preconditions ;
39
54
40
55
import java .util .ArrayList ;
56
+ import java .util .Collection ;
41
57
import java .util .List ;
42
58
import java .util .concurrent .CompletableFuture ;
43
59
import java .util .concurrent .ExecutionException ;
@@ -132,6 +148,40 @@ protected static void resetContextEnvironment() {
132
148
contextEnvironmentFactory = null ;
133
149
}
134
150
151
+ @ Override
152
+ public <OUT > NonKeyedPartitionStream <OUT > fromSource (Source <OUT > source , String sourceName ) {
153
+ if (source instanceof WrappedSource ) {
154
+ org .apache .flink .api .connector .source .Source <OUT , ?, ?> innerSource =
155
+ ((WrappedSource <OUT >) source ).getWrappedSource ();
156
+ final TypeInformation <OUT > resolvedTypeInfo =
157
+ getSourceTypeInfo (innerSource , sourceName );
158
+
159
+ SourceTransformation <OUT , ?, ?> sourceTransformation =
160
+ new SourceTransformation <>(
161
+ sourceName ,
162
+ innerSource ,
163
+ WatermarkStrategy .noWatermarks (),
164
+ resolvedTypeInfo ,
165
+ getParallelism (),
166
+ false );
167
+ return new NonKeyedPartitionStreamImpl <>(this , sourceTransformation );
168
+ } else if (source instanceof FromDataSource ) {
169
+ Collection <OUT > data = ((FromDataSource <OUT >) source ).getData ();
170
+ TypeInformation <OUT > outType = extractTypeInfoFromCollection (data );
171
+
172
+ FromElementsGeneratorFunction <OUT > generatorFunction =
173
+ new FromElementsGeneratorFunction <>(outType , executionConfig , data );
174
+
175
+ DataGeneratorSource <OUT > generatorSource =
176
+ new DataGeneratorSource <>(generatorFunction , data .size (), outType );
177
+
178
+ return fromSource (new WrappedSource <>(generatorSource ), "Collection Source" );
179
+ } else {
180
+ throw new UnsupportedOperationException (
181
+ "Unsupported type of sink, you could use DataStreamV2SourceUtils to wrap a FLIP-27 based source." );
182
+ }
183
+ }
184
+
135
185
public Configuration getConfiguration () {
136
186
return this .configuration ;
137
187
}
@@ -156,6 +206,54 @@ public void setParallelism(int parallelism) {
156
206
// Internal Methods
157
207
// -----------------------------------------------
158
208
209
+ private static <OUT > TypeInformation <OUT > extractTypeInfoFromCollection (Collection <OUT > data ) {
210
+ Preconditions .checkNotNull (data , "Collection must not be null" );
211
+ if (data .isEmpty ()) {
212
+ throw new IllegalArgumentException ("Collection must not be empty" );
213
+ }
214
+
215
+ OUT first = data .iterator ().next ();
216
+ if (first == null ) {
217
+ throw new IllegalArgumentException ("Collection must not contain null elements" );
218
+ }
219
+
220
+ TypeInformation <OUT > typeInfo ;
221
+ try {
222
+ typeInfo = TypeExtractor .getForObject (first );
223
+ } catch (Exception e ) {
224
+ throw new RuntimeException (
225
+ "Could not create TypeInformation for type "
226
+ + first .getClass ()
227
+ + "; please specify the TypeInformation manually via the version of the "
228
+ + "method that explicitly accepts it as an argument." ,
229
+ e );
230
+ }
231
+ return typeInfo ;
232
+ }
233
+
234
+ @ SuppressWarnings ("unchecked" )
235
+ private static <OUT , T extends TypeInformation <OUT >> T getSourceTypeInfo (
236
+ org .apache .flink .api .connector .source .Source <OUT , ?, ?> source , String sourceName ) {
237
+ TypeInformation <OUT > resolvedTypeInfo = null ;
238
+ if (source instanceof ResultTypeQueryable ) {
239
+ resolvedTypeInfo = ((ResultTypeQueryable <OUT >) source ).getProducedType ();
240
+ }
241
+ if (resolvedTypeInfo == null ) {
242
+ try {
243
+ resolvedTypeInfo =
244
+ TypeExtractor .createTypeInfo (
245
+ org .apache .flink .api .connector .source .Source .class ,
246
+ source .getClass (),
247
+ 0 ,
248
+ null ,
249
+ null );
250
+ } catch (final InvalidTypesException e ) {
251
+ resolvedTypeInfo = (TypeInformation <OUT >) new MissingTypeInfo (sourceName , e );
252
+ }
253
+ }
254
+ return (T ) resolvedTypeInfo ;
255
+ }
256
+
159
257
public void addOperator (Transformation <?> transformation ) {
160
258
checkNotNull (transformation , "transformation must not be null." );
161
259
this .transformations .add (transformation );
0 commit comments