28
28
import org .apache .flink .connector .jdbc .source .enumerator .SqlTemplateSplitEnumerator ;
29
29
import org .apache .flink .connector .jdbc .source .reader .extractor .ResultExtractor ;
30
30
import org .apache .flink .connector .jdbc .split .JdbcParameterValuesProvider ;
31
+ import org .apache .flink .connector .jdbc .split .JdbcSlideTimingParameterProvider ;
32
+ import org .apache .flink .connector .jdbc .utils .ContinuousEnumerationSettings ;
31
33
import org .apache .flink .types .Row ;
32
34
import org .apache .flink .util .Preconditions ;
33
35
import org .apache .flink .util .StringUtils ;
36
38
import org .slf4j .LoggerFactory ;
37
39
38
40
import javax .annotation .Nonnull ;
41
+ import javax .annotation .Nullable ;
39
42
43
+ import java .io .Serializable ;
40
44
import java .sql .DriverManager ;
41
45
import java .sql .PreparedStatement ;
46
+ import java .sql .ResultSet ;
47
+ import java .util .Objects ;
42
48
43
49
import static org .apache .flink .connector .jdbc .source .JdbcSourceOptions .AUTO_COMMIT ;
44
50
import static org .apache .flink .connector .jdbc .source .JdbcSourceOptions .READER_FETCH_BATCH_SIZE ;
@@ -94,6 +100,11 @@ public class JdbcSourceBuilder<OUT> {
94
100
95
101
public static final Logger LOG = LoggerFactory .getLogger (JdbcSourceBuilder .class );
96
102
103
+ public static final String INVALID_CONTINUOUS_SLIDE_TIMING_HINT =
104
+ "The 'jdbcParameterValuesProvider' must be specified with in type of 'JdbcSlideTimingParameterProvider' when using 'continuousEnumerationSettings'." ;
105
+ public static final String INVALID_SLIDE_TIMING_CONTINUOUS_HINT =
106
+ "The 'continuousEnumerationSettings' must be specified with in type of 'ContinuousEnumerationSettings' when using 'jdbcParameterValuesProvider' in type of 'JdbcSlideTimingParameterProvider'." ;
107
+
97
108
private final Configuration configuration ;
98
109
99
110
private int splitReaderFetchBatchSize ;
@@ -103,15 +114,15 @@ public class JdbcSourceBuilder<OUT> {
103
114
// Boolean to distinguish between default value and explicitly set autoCommit mode.
104
115
private Boolean autoCommit ;
105
116
106
- // TODO It would be used to introduce streaming semantic and tracked in
107
- // https://issues.apache.org/jira/browse/FLINK-33461
108
117
private DeliveryGuarantee deliveryGuarantee ;
118
+ private @ Nullable ContinuousEnumerationSettings continuousEnumerationSettings ;
109
119
110
120
private TypeInformation <OUT > typeInformation ;
111
121
112
122
private final JdbcConnectionOptions .JdbcConnectionOptionsBuilder connOptionsBuilder ;
113
123
private String sql ;
114
124
private JdbcParameterValuesProvider jdbcParameterValuesProvider ;
125
+ private @ Nullable Serializable optionalSqlSplitEnumeratorState ;
115
126
private ResultExtractor <OUT > resultExtractor ;
116
127
117
128
private JdbcConnectionProvider connectionProvider ;
@@ -177,6 +188,32 @@ public JdbcSourceBuilder<OUT> setTypeInformation(
177
188
178
189
// ------ Optional ------------------------------------------------------------------
179
190
191
+ /**
192
+ * The continuousEnumerationSettings to discovery the next available batch splits. Note: If the
193
+ * value was set, the {@link #jdbcParameterValuesProvider} must specified with the {@link
194
+ * org.apache.flink.connector.jdbc.split.JdbcSlideTimingParameterProvider}.
195
+ */
196
+ public JdbcSourceBuilder <OUT > setContinuousEnumerationSettings (
197
+ ContinuousEnumerationSettings continuousEnumerationSettings ) {
198
+ this .continuousEnumerationSettings = continuousEnumerationSettings ;
199
+ return this ;
200
+ }
201
+
202
+ /**
203
+ * If the value was set as an instance of {@link JdbcSlideTimingParameterProvider}, it's
204
+ * required to specify the {@link #continuousEnumerationSettings}.
205
+ */
206
+ public JdbcSourceBuilder <OUT > setJdbcParameterValuesProvider (
207
+ @ Nonnull JdbcParameterValuesProvider parameterValuesProvider ) {
208
+ this .jdbcParameterValuesProvider = Preconditions .checkNotNull (parameterValuesProvider );
209
+ return this ;
210
+ }
211
+
212
+ public JdbcSourceBuilder <OUT > setDeliveryGuarantee (DeliveryGuarantee deliveryGuarantee ) {
213
+ this .deliveryGuarantee = Preconditions .checkNotNull (deliveryGuarantee );
214
+ return this ;
215
+ }
216
+
180
217
public JdbcSourceBuilder <OUT > setConnectionCheckTimeoutSeconds (
181
218
int connectionCheckTimeoutSeconds ) {
182
219
connOptionsBuilder .withConnectionCheckTimeoutSeconds (connectionCheckTimeoutSeconds );
@@ -190,12 +227,6 @@ public JdbcSourceBuilder<OUT> setConnectionProperty(String propKey, String propV
190
227
return this ;
191
228
}
192
229
193
- public JdbcSourceBuilder <OUT > setJdbcParameterValuesProvider (
194
- @ Nonnull JdbcParameterValuesProvider parameterValuesProvider ) {
195
- this .jdbcParameterValuesProvider = Preconditions .checkNotNull (parameterValuesProvider );
196
- return this ;
197
- }
198
-
199
230
public JdbcSourceBuilder <OUT > setSplitReaderFetchBatchSize (int splitReaderFetchBatchSize ) {
200
231
Preconditions .checkArgument (
201
232
splitReaderFetchBatchSize > 0 ,
@@ -235,11 +266,26 @@ public JdbcSourceBuilder<OUT> setConnectionProvider(
235
266
return this ;
236
267
}
237
268
269
+ public JdbcSourceBuilder <OUT > setOptionalSqlSplitEnumeratorState (
270
+ Serializable optionalSqlSplitEnumeratorState ) {
271
+ this .optionalSqlSplitEnumeratorState = optionalSqlSplitEnumeratorState ;
272
+ return this ;
273
+ }
274
+
238
275
public JdbcSource <OUT > build () {
239
276
this .connectionProvider = new SimpleJdbcConnectionProvider (connOptionsBuilder .build ());
240
277
if (resultSetFetchSize > 0 ) {
241
278
this .configuration .set (RESULTSET_FETCH_SIZE , resultSetFetchSize );
242
279
}
280
+
281
+ if (deliveryGuarantee == DeliveryGuarantee .EXACTLY_ONCE ) {
282
+ Preconditions .checkArgument (
283
+ this .resultSetType == ResultSet .TYPE_SCROLL_INSENSITIVE
284
+ || this .resultSetType == ResultSet .CONCUR_READ_ONLY ,
285
+ "The 'resultSetType' must be ResultSet.TYPE_SCROLL_INSENSITIVE or ResultSet.CONCUR_READ_ONLY when using %s" ,
286
+ DeliveryGuarantee .EXACTLY_ONCE );
287
+ }
288
+
243
289
this .configuration .set (RESULTSET_CONCURRENCY , resultSetConcurrency );
244
290
this .configuration .set (RESULTSET_TYPE , resultSetType );
245
291
this .configuration .set (READER_FETCH_BATCH_SIZE , splitReaderFetchBatchSize );
@@ -250,15 +296,31 @@ public JdbcSource<OUT> build() {
250
296
Preconditions .checkNotNull (resultExtractor , "'resultExtractor' mustn't be null." );
251
297
Preconditions .checkNotNull (typeInformation , "'typeInformation' mustn't be null." );
252
298
299
+ if (Objects .nonNull (continuousEnumerationSettings )) {
300
+ Preconditions .checkArgument (
301
+ Objects .nonNull (jdbcParameterValuesProvider )
302
+ && jdbcParameterValuesProvider
303
+ instanceof JdbcSlideTimingParameterProvider ,
304
+ INVALID_SLIDE_TIMING_CONTINUOUS_HINT );
305
+ }
306
+
307
+ if (Objects .nonNull (jdbcParameterValuesProvider )
308
+ && jdbcParameterValuesProvider instanceof JdbcSlideTimingParameterProvider ) {
309
+ Preconditions .checkArgument (
310
+ Objects .nonNull (continuousEnumerationSettings ),
311
+ INVALID_CONTINUOUS_SLIDE_TIMING_HINT );
312
+ }
313
+
253
314
return new JdbcSource <>(
254
315
configuration ,
255
316
connectionProvider ,
256
317
new SqlTemplateSplitEnumerator .TemplateSqlSplitEnumeratorProvider ()
257
- .setOptionalSqlSplitEnumeratorState (null )
318
+ .setOptionalSqlSplitEnumeratorState (optionalSqlSplitEnumeratorState )
258
319
.setSqlTemplate (sql )
259
320
.setParameterValuesProvider (jdbcParameterValuesProvider ),
260
321
resultExtractor ,
261
322
typeInformation ,
262
- deliveryGuarantee );
323
+ deliveryGuarantee ,
324
+ continuousEnumerationSettings );
263
325
}
264
326
}
0 commit comments