24
24
import org .apache .flink .api .common .operators .SlotSharingGroup ;
25
25
import org .apache .flink .api .common .typeinfo .TypeInformation ;
26
26
import org .apache .flink .api .connector .sink2 .Sink ;
27
- import org .apache .flink .api .connector .sink2 .TwoPhaseCommittingSink ;
27
+ import org .apache .flink .api .connector .sink2 .SupportsCommitter ;
28
28
import org .apache .flink .api .dag .Transformation ;
29
29
import org .apache .flink .configuration .CoreOptions ;
30
30
import org .apache .flink .streaming .api .connector .sink2 .CommittableMessage ;
31
31
import org .apache .flink .streaming .api .connector .sink2 .CommittableMessageTypeInfo ;
32
32
import org .apache .flink .streaming .api .connector .sink2 .StandardSinkTopologies ;
33
- import org .apache .flink .streaming .api .connector .sink2 .WithPostCommitTopology ;
34
- import org .apache .flink .streaming .api .connector .sink2 .WithPreCommitTopology ;
35
- import org .apache .flink .streaming .api .connector .sink2 .WithPreWriteTopology ;
33
+ import org .apache .flink .streaming .api .connector .sink2 .SupportsPostCommitTopology ;
34
+ import org .apache .flink .streaming .api .connector .sink2 .SupportsPreCommitTopology ;
35
+ import org .apache .flink .streaming .api .connector .sink2 .SupportsPreWriteTopology ;
36
36
import org .apache .flink .streaming .api .datastream .CustomSinkOperatorUidHashes ;
37
37
import org .apache .flink .streaming .api .datastream .DataStream ;
38
38
import org .apache .flink .streaming .api .environment .StreamExecutionEnvironment ;
44
44
import org .apache .flink .streaming .runtime .operators .sink .CommitterOperatorFactory ;
45
45
import org .apache .flink .streaming .runtime .operators .sink .SinkWriterOperatorFactory ;
46
46
import org .apache .flink .streaming .runtime .partitioner .ForwardPartitioner ;
47
+ import org .apache .flink .util .Preconditions ;
47
48
48
49
import javax .annotation .Nullable ;
49
50
@@ -135,16 +136,27 @@ private void expand() {
135
136
136
137
DataStream <T > prewritten = inputStream ;
137
138
138
- if (sink instanceof WithPreWriteTopology ) {
139
+ if (sink instanceof SupportsPreWriteTopology ) {
139
140
prewritten =
140
141
adjustTransformations (
141
142
prewritten ,
142
- ((WithPreWriteTopology <T >) sink )::addPreWriteTopology ,
143
+ ((SupportsPreWriteTopology <T >) sink )::addPreWriteTopology ,
143
144
true ,
144
145
sink instanceof SupportsConcurrentExecutionAttempts );
145
146
}
146
147
147
- if (sink instanceof TwoPhaseCommittingSink ) {
148
+ if (sink instanceof SupportsPreCommitTopology ) {
149
+ Preconditions .checkArgument (
150
+ sink instanceof SupportsCommitter ,
151
+ "Sink with SupportsPreCommitTopology should implement SupportsCommitter" );
152
+ }
153
+ if (sink instanceof SupportsPostCommitTopology ) {
154
+ Preconditions .checkArgument (
155
+ sink instanceof SupportsCommitter ,
156
+ "Sink with SupportsPostCommitTopology should implement SupportsCommitter" );
157
+ }
158
+
159
+ if (sink instanceof SupportsCommitter ) {
148
160
addCommittingTopology (sink , prewritten );
149
161
} else {
150
162
adjustTransformations (
@@ -173,32 +185,27 @@ private void expand() {
173
185
}
174
186
}
175
187
176
- private <CommT > void addCommittingTopology (Sink < T > sink , DataStream < T > inputStream ) {
177
- TwoPhaseCommittingSink < T , CommT > committingSink =
178
- ( TwoPhaseCommittingSink < T , CommT >) sink ;
179
- TypeInformation <CommittableMessage <CommT >> typeInformation =
188
+ private <CommT , WriteResultT > void addCommittingTopology (
189
+ Sink < T > sink , DataStream < T > inputStream ) {
190
+ SupportsCommitter < CommT > committingSink = ( SupportsCommitter < CommT >) sink ;
191
+ TypeInformation <CommittableMessage <CommT >> committableTypeInformation =
180
192
CommittableMessageTypeInfo .of (committingSink ::getCommittableSerializer );
181
193
182
- DataStream <CommittableMessage <CommT >> written =
183
- adjustTransformations (
184
- inputStream ,
185
- input ->
186
- input .transform (
187
- WRITER_NAME ,
188
- typeInformation ,
189
- new SinkWriterOperatorFactory <>(sink )),
190
- false ,
191
- sink instanceof SupportsConcurrentExecutionAttempts );
194
+ DataStream <CommittableMessage <CommT >> precommitted ;
195
+ if (sink instanceof SupportsPreCommitTopology ) {
196
+ SupportsPreCommitTopology <WriteResultT , CommT > preCommittingSink =
197
+ (SupportsPreCommitTopology <WriteResultT , CommT >) sink ;
198
+ TypeInformation <CommittableMessage <WriteResultT >> writeResultTypeInformation =
199
+ CommittableMessageTypeInfo .of (preCommittingSink ::getWriteResultSerializer );
192
200
193
- DataStream <CommittableMessage <CommT >> precommitted = addFailOverRegion (written );
201
+ DataStream <CommittableMessage <WriteResultT >> writerResult =
202
+ addWriter (sink , inputStream , writeResultTypeInformation );
194
203
195
- if (sink instanceof WithPreCommitTopology ) {
196
204
precommitted =
197
205
adjustTransformations (
198
- precommitted ,
199
- ((WithPreCommitTopology <T , CommT >) sink )::addPreCommitTopology ,
200
- true ,
201
- false );
206
+ writerResult , preCommittingSink ::addPreCommitTopology , true , false );
207
+ } else {
208
+ precommitted = addWriter (sink , inputStream , committableTypeInformation );
202
209
}
203
210
204
211
DataStream <CommittableMessage <CommT >> committed =
@@ -207,27 +214,45 @@ private <CommT> void addCommittingTopology(Sink<T> sink, DataStream<T> inputStre
207
214
pc ->
208
215
pc .transform (
209
216
COMMITTER_NAME ,
210
- typeInformation ,
217
+ committableTypeInformation ,
211
218
new CommitterOperatorFactory <>(
212
219
committingSink ,
213
220
isBatchMode ,
214
221
isCheckpointingEnabled )),
215
222
false ,
216
223
false );
217
224
218
- if (sink instanceof WithPostCommitTopology ) {
225
+ if (sink instanceof SupportsPostCommitTopology ) {
219
226
DataStream <CommittableMessage <CommT >> postcommitted = addFailOverRegion (committed );
220
227
adjustTransformations (
221
228
postcommitted ,
222
229
pc -> {
223
- ((WithPostCommitTopology < T , CommT >) sink ).addPostCommitTopology (pc );
230
+ ((SupportsPostCommitTopology < CommT >) sink ).addPostCommitTopology (pc );
224
231
return null ;
225
232
},
226
233
true ,
227
234
false );
228
235
}
229
236
}
230
237
238
+ private <WriteResultT > DataStream <CommittableMessage <WriteResultT >> addWriter (
239
+ Sink <T > sink ,
240
+ DataStream <T > inputStream ,
241
+ TypeInformation <CommittableMessage <WriteResultT >> typeInformation ) {
242
+ DataStream <CommittableMessage <WriteResultT >> written =
243
+ adjustTransformations (
244
+ inputStream ,
245
+ input ->
246
+ input .transform (
247
+ WRITER_NAME ,
248
+ typeInformation ,
249
+ new SinkWriterOperatorFactory <>(sink )),
250
+ false ,
251
+ sink instanceof SupportsConcurrentExecutionAttempts );
252
+
253
+ return addFailOverRegion (written );
254
+ }
255
+
231
256
/**
232
257
* Adds a batch exchange that materializes the output first. This is a no-op in STREAMING.
233
258
*/
0 commit comments