|
19 | 19 | package org.apache.flink.datastream.impl.utils;
|
20 | 20 |
|
21 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation;
|
| 22 | +import org.apache.flink.api.connector.dsv2.Sink; |
| 23 | +import org.apache.flink.api.connector.dsv2.WrappedSink; |
22 | 24 | import org.apache.flink.api.java.Utils;
|
23 | 25 | import org.apache.flink.api.java.functions.KeySelector;
|
24 | 26 | import org.apache.flink.api.java.tuple.Tuple2;
|
|
29 | 31 | import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction;
|
30 | 32 | import org.apache.flink.datastream.impl.stream.AbstractDataStream;
|
31 | 33 | import org.apache.flink.datastream.impl.stream.KeyedPartitionStreamImpl;
|
| 34 | +import org.apache.flink.datastream.impl.stream.NonKeyedPartitionStreamImpl; |
32 | 35 | import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
|
33 | 36 | import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
|
34 | 37 | import org.apache.flink.streaming.api.operators.SimpleUdfStreamOperatorFactory;
|
| 38 | +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; |
35 | 39 | import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
|
| 40 | +import org.apache.flink.streaming.api.transformations.DataStreamV2SinkTransformation; |
36 | 41 | import org.apache.flink.streaming.api.transformations.OneInputTransformation;
|
37 | 42 | import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
|
38 | 43 |
|
@@ -228,4 +233,56 @@ public static <IN1, IN2, OUT> TwoInputTransformation<IN1, IN2, OUT> getTwoInputT
|
228 | 233 |
|
229 | 234 | return transform;
|
230 | 235 | }
|
| 236 | + |
| 237 | + /** Construct and return a new DataStream with one input operator. */ |
| 238 | + public static <T, R> AbstractDataStream<R> transformOneInputOperator( |
| 239 | + String operatorName, |
| 240 | + AbstractDataStream<T> inputStream, |
| 241 | + TypeInformation<R> outTypeInfo, |
| 242 | + StreamOperatorFactory<R> operatorFactory) { |
| 243 | + // read the output type of the input Transform to coax out errors about MissingTypeInfo |
| 244 | + inputStream.getTransformation().getOutputType(); |
| 245 | + |
| 246 | + OneInputTransformation<T, R> resultTransform = |
| 247 | + new OneInputTransformation<>( |
| 248 | + inputStream.getTransformation(), |
| 249 | + operatorName, |
| 250 | + operatorFactory, |
| 251 | + outTypeInfo, |
| 252 | + inputStream.getEnvironment().getParallelism(), |
| 253 | + false); |
| 254 | + |
| 255 | + NonKeyedPartitionStreamImpl<R> returnStream = |
| 256 | + new NonKeyedPartitionStreamImpl<>(inputStream.getEnvironment(), resultTransform); |
| 257 | + |
| 258 | + inputStream.getEnvironment().addOperator(resultTransform); |
| 259 | + |
| 260 | + return returnStream; |
| 261 | + } |
| 262 | + |
| 263 | + /** Add sink operator to the input stream. */ |
| 264 | + public static <T> DataStreamV2SinkTransformation<T, T> addSinkOperator( |
| 265 | + AbstractDataStream<T> inputStream, Sink<T> sink, TypeInformation<T> typeInformation) { |
| 266 | + // read the output type of the input Transform to coax out errors about MissingTypeInfo |
| 267 | + inputStream.getTransformation().getOutputType(); |
| 268 | + |
| 269 | + if (!(sink instanceof WrappedSink)) { |
| 270 | + throw new UnsupportedOperationException( |
| 271 | + "Unsupported type of sink, please use DataStreamV2SinkUtils to wrap a sink-v2 sink first."); |
| 272 | + } |
| 273 | + |
| 274 | + org.apache.flink.api.connector.sink2.Sink<T> innerSink = |
| 275 | + ((WrappedSink<T>) sink).getWrappedSink(); |
| 276 | + |
| 277 | + DataStreamV2SinkTransformation<T, T> sinkTransformation = |
| 278 | + new DataStreamV2SinkTransformation<>( |
| 279 | + inputStream, |
| 280 | + innerSink, |
| 281 | + typeInformation, |
| 282 | + "Sink", |
| 283 | + inputStream.getEnvironment().getParallelism(), |
| 284 | + false); |
| 285 | + inputStream.getEnvironment().addOperator(sinkTransformation); |
| 286 | + return sinkTransformation; |
| 287 | + } |
231 | 288 | }
|
0 commit comments