forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
/
JdbcDynamicTableFactory.java
415 lines (378 loc) · 19.3 KB
/
JdbcDynamicTableFactory.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.connector.jdbc.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.factories.DynamicTableSinkFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.utils.TableSchemaUtils;
import org.apache.flink.util.Preconditions;
import java.time.Duration;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import java.util.regex.Pattern;
import static org.apache.flink.util.Preconditions.checkState;
/**
* Factory for creating configured instances of {@link JdbcDynamicTableSource} and {@link
* JdbcDynamicTableSink}.
*/
@Internal
public class JdbcDynamicTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
public static final String IDENTIFIER = "jdbc";
public static final ConfigOption<String> URL =
ConfigOptions.key("url")
.stringType()
.noDefaultValue()
.withDescription("the jdbc database url.");
public static final ConfigOption<String> TABLE_NAME =
ConfigOptions.key("table-name")
.stringType()
.noDefaultValue()
.withDescription("the jdbc table name. Either 'table-name' or 'table-pattern' must be set.");
public static final ConfigOption<String> TABLE_PATTERN =
ConfigOptions.key("table-pattern")
.stringType()
.noDefaultValue()
.withDescription(
"Optional table pattern from which the table is read for source. Either 'table-name' or 'table-pattern' must be set.");
public static final ConfigOption<String> USERNAME =
ConfigOptions.key("username")
.stringType()
.noDefaultValue()
.withDescription("the jdbc user name.");
public static final ConfigOption<String> PASSWORD =
ConfigOptions.key("password")
.stringType()
.noDefaultValue()
.withDescription("the jdbc password.");
private static final ConfigOption<String> DRIVER =
ConfigOptions.key("driver")
.stringType()
.noDefaultValue()
.withDescription(
"the class name of the JDBC driver to use to connect to this URL. "
+ "If not set, it will automatically be derived from the URL.");
public static final ConfigOption<Duration> MAX_RETRY_TIMEOUT =
ConfigOptions.key("connection.max-retry-timeout")
.durationType()
.defaultValue(Duration.ofSeconds(60))
.withDescription("Maximum timeout between retries.");
// read config options
private static final ConfigOption<String> SCAN_PARTITION_COLUMN =
ConfigOptions.key("scan.partition.column")
.stringType()
.noDefaultValue()
.withDescription("the column name used for partitioning the input.");
private static final ConfigOption<Integer> SCAN_PARTITION_NUM =
ConfigOptions.key("scan.partition.num")
.intType()
.noDefaultValue()
.withDescription("the number of partitions.");
private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND =
ConfigOptions.key("scan.partition.lower-bound")
.longType()
.noDefaultValue()
.withDescription("the smallest value of the first partition.");
private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND =
ConfigOptions.key("scan.partition.upper-bound")
.longType()
.noDefaultValue()
.withDescription("the largest value of the last partition.");
private static final ConfigOption<Integer> SCAN_FETCH_SIZE =
ConfigOptions.key("scan.fetch-size")
.intType()
.defaultValue(0)
.withDescription(
"gives the reader a hint as to the number of rows that should be fetched, from"
+ " the database when reading per round trip. If the value specified is zero, then the hint is ignored. The"
+ " default value is zero.");
private static final ConfigOption<Boolean> SCAN_AUTO_COMMIT =
ConfigOptions.key("scan.auto-commit")
.booleanType()
.defaultValue(true)
.withDescription(
"sets whether the driver is in auto-commit mode. The default value is true, per"
+ " the JDBC spec.");
// look up config options
private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS =
ConfigOptions.key("lookup.cache.max-rows")
.longType()
.defaultValue(-1L)
.withDescription(
"the max number of rows of lookup cache, over this value, the oldest rows will "
+ "be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is "
+ "specified. Cache is not enabled as default.");
private static final ConfigOption<Duration> LOOKUP_CACHE_TTL =
ConfigOptions.key("lookup.cache.ttl")
.durationType()
.defaultValue(Duration.ofSeconds(10))
.withDescription("the cache time to live.");
private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES =
ConfigOptions.key("lookup.max-retries")
.intType()
.defaultValue(3)
.withDescription("the max retry times if lookup database failed.");
// write config options
private static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
ConfigOptions.key("sink.buffer-flush.max-rows")
.intType()
.defaultValue(100)
.withDescription(
"the flush max size (includes all append, upsert and delete records), over this number"
+ " of records, will flush data. The default value is 100.");
private static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
ConfigOptions.key("sink.buffer-flush.interval")
.durationType()
.defaultValue(Duration.ofSeconds(1))
.withDescription(
"the flush interval mills, over this time, asynchronous threads will flush data. The "
+ "default value is 1s.");
private static final ConfigOption<Integer> SINK_MAX_RETRIES =
ConfigOptions.key("sink.max-retries")
.intType()
.defaultValue(3)
.withDescription("the max retry times if writing records to database failed.");
@Override
public DynamicTableSink createDynamicTableSink(Context context) {
final FactoryUtil.TableFactoryHelper helper =
FactoryUtil.createTableFactoryHelper(this, context);
final ReadableConfig config = helper.getOptions();
helper.validate();
validateConfigOptions(config);
validateSinkAndLookUpTable(config);
JdbcOptions jdbcOptions = getJdbcOptions(config);
TableSchema physicalSchema =
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
return new JdbcDynamicTableSink(
jdbcOptions,
getJdbcExecutionOptions(config),
getJdbcDmlOptions(jdbcOptions, physicalSchema),
physicalSchema);
}
@Override
public DynamicTableSource createDynamicTableSource(Context context) {
final FactoryUtil.TableFactoryHelper helper =
FactoryUtil.createTableFactoryHelper(this, context);
final ReadableConfig config = helper.getOptions();
helper.validate();
validateConfigOptions(config);
validateSourceTable(config);
TableSchema physicalSchema =
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
return new JdbcDynamicTableSource(
getJdbcOptions(helper.getOptions()),
getJdbcReadOptions(helper.getOptions()),
getJdbcLookupOptions(helper.getOptions()),
physicalSchema);
}
private JdbcOptions getJdbcOptions(ReadableConfig readableConfig) {
final String url = readableConfig.get(URL);
final JdbcOptions.Builder builder =
JdbcOptions.builder()
.setDBUrl(url)
.setTableName(readableConfig.get(TABLE_NAME))
.setTablePattern(readableConfig.getOptional(TABLE_PATTERN).map(Pattern::compile).orElse(null))
.setDialect(JdbcDialects.get(url).get())
.setParallelism(
readableConfig
.getOptional(FactoryUtil.SINK_PARALLELISM)
.orElse(null))
.setConnectionCheckTimeoutSeconds(
(int) readableConfig.get(MAX_RETRY_TIMEOUT).getSeconds());
readableConfig.getOptional(DRIVER).ifPresent(builder::setDriverName);
readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername);
readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword);
return builder.build();
}
private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) {
final Optional<String> partitionColumnName =
readableConfig.getOptional(SCAN_PARTITION_COLUMN);
final JdbcReadOptions.Builder builder = JdbcReadOptions.builder();
if (partitionColumnName.isPresent()) {
builder.setPartitionColumnName(partitionColumnName.get());
builder.setPartitionLowerBound(readableConfig.get(SCAN_PARTITION_LOWER_BOUND));
builder.setPartitionUpperBound(readableConfig.get(SCAN_PARTITION_UPPER_BOUND));
builder.setNumPartitions(readableConfig.get(SCAN_PARTITION_NUM));
}
readableConfig.getOptional(SCAN_FETCH_SIZE).ifPresent(builder::setFetchSize);
builder.setAutoCommit(readableConfig.get(SCAN_AUTO_COMMIT));
return builder.build();
}
private JdbcLookupOptions getJdbcLookupOptions(ReadableConfig readableConfig) {
return new JdbcLookupOptions(
readableConfig.get(LOOKUP_CACHE_MAX_ROWS),
readableConfig.get(LOOKUP_CACHE_TTL).toMillis(),
readableConfig.get(LOOKUP_MAX_RETRIES));
}
private JdbcExecutionOptions getJdbcExecutionOptions(ReadableConfig config) {
final JdbcExecutionOptions.Builder builder = new JdbcExecutionOptions.Builder();
builder.withBatchSize(config.get(SINK_BUFFER_FLUSH_MAX_ROWS));
builder.withBatchIntervalMs(config.get(SINK_BUFFER_FLUSH_INTERVAL).toMillis());
builder.withMaxRetries(config.get(SINK_MAX_RETRIES));
return builder.build();
}
private JdbcDmlOptions getJdbcDmlOptions(JdbcOptions jdbcOptions, TableSchema schema) {
String[] keyFields =
schema.getPrimaryKey()
.map(pk -> pk.getColumns().toArray(new String[0]))
.orElse(null);
return JdbcDmlOptions.builder()
.withTableName(jdbcOptions.getTableName())
.withDialect(jdbcOptions.getDialect())
.withFieldNames(schema.getFieldNames())
.withKeyFields(keyFields)
.build();
}
@Override
public String factoryIdentifier() {
return IDENTIFIER;
}
@Override
public Set<ConfigOption<?>> requiredOptions() {
Set<ConfigOption<?>> requiredOptions = new HashSet<>();
requiredOptions.add(URL);
return requiredOptions;
}
@Override
public Set<ConfigOption<?>> optionalOptions() {
Set<ConfigOption<?>> optionalOptions = new HashSet<>();
optionalOptions.add(DRIVER);
optionalOptions.add(USERNAME);
optionalOptions.add(PASSWORD);
optionalOptions.add(TABLE_NAME);
optionalOptions.add(TABLE_PATTERN);
optionalOptions.add(SCAN_PARTITION_COLUMN);
optionalOptions.add(SCAN_PARTITION_LOWER_BOUND);
optionalOptions.add(SCAN_PARTITION_UPPER_BOUND);
optionalOptions.add(SCAN_PARTITION_NUM);
optionalOptions.add(SCAN_FETCH_SIZE);
optionalOptions.add(SCAN_AUTO_COMMIT);
optionalOptions.add(LOOKUP_CACHE_MAX_ROWS);
optionalOptions.add(LOOKUP_CACHE_TTL);
optionalOptions.add(LOOKUP_MAX_RETRIES);
optionalOptions.add(SINK_BUFFER_FLUSH_MAX_ROWS);
optionalOptions.add(SINK_BUFFER_FLUSH_INTERVAL);
optionalOptions.add(SINK_MAX_RETRIES);
optionalOptions.add(FactoryUtil.SINK_PARALLELISM);
optionalOptions.add(MAX_RETRY_TIMEOUT);
return optionalOptions;
}
private void validateConfigOptions(ReadableConfig config) {
String jdbcUrl = config.get(URL);
final Optional<JdbcDialect> dialect = JdbcDialects.get(jdbcUrl);
checkState(dialect.isPresent(), "Cannot handle such jdbc url: " + jdbcUrl);
checkAllOrNone(config, new ConfigOption[] {USERNAME, PASSWORD});
checkAllOrNone(
config,
new ConfigOption[] {
SCAN_PARTITION_COLUMN,
SCAN_PARTITION_NUM,
SCAN_PARTITION_LOWER_BOUND,
SCAN_PARTITION_UPPER_BOUND
});
if (config.getOptional(SCAN_PARTITION_LOWER_BOUND).isPresent()
&& config.getOptional(SCAN_PARTITION_UPPER_BOUND).isPresent()) {
long lowerBound = config.get(SCAN_PARTITION_LOWER_BOUND);
long upperBound = config.get(SCAN_PARTITION_UPPER_BOUND);
if (lowerBound > upperBound) {
throw new IllegalArgumentException(
String.format(
"'%s'='%s' must not be larger than '%s'='%s'.",
SCAN_PARTITION_LOWER_BOUND.key(),
lowerBound,
SCAN_PARTITION_UPPER_BOUND.key(),
upperBound));
}
}
checkAllOrNone(config, new ConfigOption[] {LOOKUP_CACHE_MAX_ROWS, LOOKUP_CACHE_TTL});
if (config.get(LOOKUP_MAX_RETRIES) < 0) {
throw new IllegalArgumentException(
String.format(
"The value of '%s' option shouldn't be negative, but is %s.",
LOOKUP_MAX_RETRIES.key(), config.get(LOOKUP_MAX_RETRIES)));
}
if (config.get(SINK_MAX_RETRIES) < 0) {
throw new IllegalArgumentException(
String.format(
"The value of '%s' option shouldn't be negative, but is %s.",
SINK_MAX_RETRIES.key(), config.get(SINK_MAX_RETRIES)));
}
if (config.get(MAX_RETRY_TIMEOUT).getSeconds() <= 0) {
throw new IllegalArgumentException(
String.format(
"The value of '%s' option must be in second granularity and shouldn't be smaller than 1 second, but is %s.",
MAX_RETRY_TIMEOUT.key(),
config.get(
ConfigOptions.key(MAX_RETRY_TIMEOUT.key())
.stringType()
.noDefaultValue())));
}
}
private void checkAllOrNone(ReadableConfig config, ConfigOption<?>[] configOptions) {
int presentCount = 0;
for (ConfigOption configOption : configOptions) {
if (config.getOptional(configOption).isPresent()) {
presentCount++;
}
}
String[] propertyNames =
Arrays.stream(configOptions).map(ConfigOption::key).toArray(String[]::new);
Preconditions.checkArgument(
configOptions.length == presentCount || presentCount == 0,
"Either all or none of the following options should be provided:\n"
+ String.join("\n", propertyNames));
}
public static void validateSourceTable(ReadableConfig config) {
Optional<String> table = config.getOptional(TABLE_NAME);
Optional<String> pattern = config.getOptional(TABLE_PATTERN);
if (table.isPresent() && pattern.isPresent()) {
throw new ValidationException(
"Option 'table-name' and 'table-pattern' shouldn't be set together.");
}
if (!table.isPresent() && !pattern.isPresent()) {
throw new ValidationException("Either 'table-name' or 'table-pattern' must be set.");
}
}
public static void validateSinkAndLookUpTable(ReadableConfig config) {
String errorMessageTemp =
"Flink Jdbc sink currently only supports table name, but got %s: %s.";
if (config.getOptional(TABLE_PATTERN).isPresent()) {
throw new ValidationException(
String.format(
errorMessageTemp,
"'table-pattern'",
config.get(TABLE_PATTERN)));
}
}
}