SCAN_PARTITION_NUM =
94 | ConfigOptions.key("scan.partition.num")
95 | .intType()
96 | .noDefaultValue()
97 | .withDescription("The number of partitions for one table.");
98 |
99 | /**
100 | * Creates a {@link DynamicTableSource} instance from a {@link CatalogTable} and additional
101 | * context information.
102 | *
103 | * An implementation should perform validation and the discovery of further (nested)
104 | * factories in this method.
105 | *
106 | * @param context
107 | */
108 | @Override
109 | public DynamicTableSource createDynamicTableSource(Context context) {
110 | context.getConfiguration();
111 | final FactoryUtil.TableFactoryHelper helper =
112 | FactoryUtil.createTableFactoryHelper(this, context);
113 | final ReadableConfig config = helper.getOptions();
114 |
115 | helper.validate();
116 | return new MultiJdbcDynamicTableSource(
117 | getJdbcOptions(helper.getOptions()),
118 | getJdbcReadOptions(helper.getOptions()),
119 | context.getCatalogTable().getResolvedSchema(),
120 | config);
121 | }
122 |
123 | /**
124 | * Returns a unique identifier among same factory interfaces.
125 | *
126 | *
For consistency, an identifier should be declared as one lower case word (e.g. {@code
127 | * kafka}). If multiple factories exist for different versions, a version should be appended
128 | * using "-" (e.g. {@code elasticsearch-7}).
129 | */
130 | @Override
131 | public String factoryIdentifier() {
132 | return IDENTIFIER;
133 | }
134 |
135 | /**
136 | * Returns a set of {@link ConfigOption} that an implementation of this factory requires in
137 | * addition to {@link #optionalOptions()}.
138 | *
139 | *
See the documentation of {@link Factory} for more information.
140 | */
141 | @Override
142 | public Set> requiredOptions() {
143 | Set> requiredOptions = new HashSet<>();
144 | requiredOptions.add(URL);
145 | requiredOptions.add(TABLE_NAME);
146 | requiredOptions.add(SCHEMA_NAME);
147 | return requiredOptions;
148 | }
149 |
150 | /**
151 | * Returns a set of {@link ConfigOption} that an implementation of this factory consumes in
152 | * addition to {@link #requiredOptions()}.
153 | *
154 | * See the documentation of {@link Factory} for more information.
155 | */
156 | @Override
157 | public Set> optionalOptions() {
158 | Set> optionalOptions = new HashSet<>();
159 | optionalOptions.add(DRIVER);
160 | optionalOptions.add(USERNAME);
161 | optionalOptions.add(PASSWORD);
162 | optionalOptions.add(SCAN_PARTITION_COLUMN);
163 | optionalOptions.add(SCAN_BATCH_SIZE);
164 | optionalOptions.add(SCAN_PARTITION_NUM);
165 | optionalOptions.add(MAX_RETRY_TIMEOUT);
166 | return optionalOptions;
167 | }
168 |
169 | private MultiJdbcOptions getJdbcOptions(ReadableConfig readableConfig) {
170 | final String url = readableConfig.get(URL);
171 | final MultiJdbcOptions.Builder builder =
172 | MultiJdbcOptions.builder()
173 | .setDBUrl(url)
174 | .setTableName(readableConfig.get(TABLE_NAME))
175 | .setSchemaName(readableConfig.get(SCHEMA_NAME))
176 | .setDialect(JdbcDialects.get(url).get())
177 | .setConnectionCheckTimeoutSeconds(
178 | (int) readableConfig.get(MAX_RETRY_TIMEOUT).getSeconds());
179 |
180 | readableConfig.getOptional(DRIVER).ifPresent(builder::setDriverName);
181 | readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername);
182 | readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword);
183 | return builder.build();
184 | }
185 |
186 | private MultiJdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) {
187 | final Optional partitionColumnName =
188 | readableConfig.getOptional(SCAN_PARTITION_COLUMN);
189 | final MultiJdbcReadOptions.Builder builder = MultiJdbcReadOptions.builder();
190 | if (partitionColumnName.isPresent()) {
191 | builder.setPartitionColumnName(partitionColumnName.get());
192 | final Optional batchSize = readableConfig.getOptional(SCAN_BATCH_SIZE);
193 | final Optional partitionNum = readableConfig.getOptional(SCAN_PARTITION_NUM);
194 | if (!batchSize.isPresent() && !partitionNum.isPresent()) {
195 | throw new RuntimeException("neither batch size or partition num is set");
196 | } else if (batchSize.isPresent() && partitionNum.isPresent()) {
197 | throw new RuntimeException("batch size and partition num can not appear currently");
198 | }
199 | partitionNum.ifPresent(builder::setNumPartitions);
200 | batchSize.ifPresent(builder::setBatchSize);
201 | }
202 | return builder.build();
203 | }
204 | }
205 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcDynamicTableSource.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.api.common.eventtime.WatermarkStrategy;
22 | import org.apache.flink.configuration.ReadableConfig;
23 | import org.apache.flink.streaming.api.datastream.DataStream;
24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
25 | import org.apache.flink.table.catalog.ResolvedSchema;
26 | import org.apache.flink.table.connector.ChangelogMode;
27 | import org.apache.flink.table.connector.source.DataStreamScanProvider;
28 | import org.apache.flink.table.connector.source.DynamicTableSource;
29 | import org.apache.flink.table.connector.source.ScanTableSource;
30 | import org.apache.flink.table.data.RowData;
31 | import org.apache.flink.types.RowKind;
32 |
33 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcOptions;
34 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcReadOptions;
35 |
36 | /** dynamic table source for multi jdbc connector. */
37 | public class MultiJdbcDynamicTableSource implements ScanTableSource {
38 | private final MultiJdbcOptions options;
39 | private final MultiJdbcReadOptions readOptions;
40 | private ResolvedSchema physicalSchema;
41 | private final ReadableConfig config;
42 |
43 | public MultiJdbcDynamicTableSource(
44 | MultiJdbcOptions options,
45 | MultiJdbcReadOptions readOptions,
46 | ResolvedSchema physicalSchema,
47 | ReadableConfig config) {
48 | this.options = options;
49 | this.readOptions = readOptions;
50 | this.physicalSchema = physicalSchema;
51 | this.config = config;
52 | }
53 |
54 | /**
55 | * Returns the set of changes that the planner can expect during runtime.
56 | *
57 | * @see RowKind
58 | */
59 | @Override
60 | public ChangelogMode getChangelogMode() {
61 | return ChangelogMode.insertOnly();
62 | }
63 |
64 | /**
65 | * Returns a provider of runtime implementation for reading the data.
66 | *
67 | * There might exist different interfaces for runtime implementation which is why {@link
68 | * ScanRuntimeProvider} serves as the base interface. Concrete {@link ScanRuntimeProvider}
69 | * interfaces might be located in other Flink modules.
70 | *
71 | *
Independent of the provider interface, the table runtime expects that a source
72 | * implementation emits internal data structures (see {@link RowData} for more information).
73 | *
74 | *
The given {@link ScanContext} offers utilities by the planner for creating runtime
75 | * implementation with minimal dependencies to internal data structures.
76 | *
77 | *
See {@code org.apache.flink.table.connector.source.SourceFunctionProvider} in {@code
78 | * flink-table-api-java-bridge}.
79 | *
80 | * @param runtimeProviderContext
81 | */
82 | @Override
83 | public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
84 | return new DataStreamScanProvider() {
85 |
86 | @Override
87 | public boolean isBounded() {
88 | return true;
89 | }
90 |
91 | @Override
92 | public DataStream produceDataStream(StreamExecutionEnvironment execEnv) {
93 | MultiJdbcSource multiJdbcSource =
94 | new MultiJdbcSource(
95 | options,
96 | readOptions,
97 | physicalSchema,
98 | config,
99 | runtimeProviderContext.createTypeInformation(
100 | physicalSchema.toPhysicalRowDataType()));
101 | return execEnv.fromSource(
102 | multiJdbcSource,
103 | WatermarkStrategy.noWatermarks(),
104 | String.format(
105 | "multi jdbc source : [%s].[%s]",
106 | options.getSchemaName(), options.getTableName()));
107 | }
108 | };
109 | }
110 |
111 | /**
112 | * Creates a copy of this instance during planning. The copy should be a deep copy of all
113 | * mutable members.
114 | */
115 | @Override
116 | public DynamicTableSource copy() {
117 | return new MultiJdbcDynamicTableSource(options, readOptions, physicalSchema, config);
118 | }
119 |
120 | /** Returns a string that summarizes this source for printing to a console or log. */
121 | @Override
122 | public String asSummaryString() {
123 | return String.format(
124 | "multi jdbc source : [%s].[%s]", options.getSchemaName(), options.getTableName());
125 | }
126 | }
127 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcEnumerator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.annotation.VisibleForTesting;
22 | import org.apache.flink.api.connector.source.SourceEvent;
23 | import org.apache.flink.api.connector.source.SourceReader;
24 | import org.apache.flink.api.connector.source.SourceReaderContext;
25 | import org.apache.flink.api.connector.source.SourceSplit;
26 | import org.apache.flink.api.connector.source.SplitEnumerator;
27 | import org.apache.flink.api.connector.source.SplitEnumeratorContext;
28 | import org.apache.flink.api.connector.source.SplitsAssignment;
29 | import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
30 | import org.slf4j.Logger;
31 | import org.slf4j.LoggerFactory;
32 |
33 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcOptions;
34 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcReadOptions;
35 |
36 | import javax.annotation.Nullable;
37 | import java.io.IOException;
38 | import java.math.BigInteger;
39 | import java.sql.Connection;
40 | import java.sql.DatabaseMetaData;
41 | import java.sql.DriverManager;
42 | import java.sql.ResultSet;
43 | import java.sql.SQLException;
44 | import java.sql.Statement;
45 | import java.util.ArrayDeque;
46 | import java.util.ArrayList;
47 | import java.util.Arrays;
48 | import java.util.Deque;
49 | import java.util.HashMap;
50 | import java.util.List;
51 | import java.util.Map;
52 | import java.util.concurrent.atomic.AtomicInteger;
53 | import java.util.regex.Matcher;
54 | import java.util.regex.Pattern;
55 | import java.util.stream.Collectors;
56 |
57 | import static java.lang.String.format;
58 |
59 | /** enumerator for multi jdbc. */
60 | public class MultiJdbcEnumerator
61 | implements SplitEnumerator {
62 |
63 | private static final Logger LOG = LoggerFactory.getLogger(MultiJdbcEnumerator.class.getName());
64 |
65 | private final MultiJdbcOptions options;
66 | private final MultiJdbcReadOptions readOptions;
67 | private final String[] columns;
68 | private final SplitEnumeratorContext context;
69 | private final Deque splitsQueue = new ArrayDeque<>();
70 |
71 | private static final String SELECT_COUNT = "select count(*) from %s";
72 | private static final String USE_DATABASE = "use %s;";
73 | private static final String SELECT_MAX_MIN = "select max(%s), min(%s) from %s";
74 |
75 | private final AtomicInteger jdbcUrlCounter = new AtomicInteger(0);
76 | private final AtomicInteger totalSchemaCounter = new AtomicInteger(0);
77 | private final AtomicInteger totalTableCounter = new AtomicInteger(0);
78 | private final AtomicInteger queryCounter = new AtomicInteger(0);
79 |
80 | public MultiJdbcEnumerator(
81 | MultiJdbcOptions options,
82 | MultiJdbcReadOptions readOptions,
83 | String[] columns,
84 | SplitEnumeratorContext context) {
85 | this.options = options;
86 | this.readOptions = readOptions;
87 | this.columns = columns;
88 | this.context = context;
89 | }
90 |
91 | /**
92 | * Start the split enumerator.
93 | *
94 | * The default behavior does nothing.
95 | */
96 | @Override
97 | public void start() {
98 | generateSplit();
99 | }
100 |
101 | /**
102 | * Handles the request for a split. This method is called when the reader with the given subtask
103 | * id calls the {@link SourceReaderContext#sendSplitRequest()} method.
104 | *
105 | * @param subtaskId the subtask id of the source reader who sent the source event.
106 | * @param requesterHostname Optional, the hostname where the requesting task is running. This
107 | */
108 | @Override
109 | public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
110 | if (!context.registeredReaders().containsKey(subtaskId)) {
111 | // reader failed between sending the request and now. skip this request.
112 | return;
113 | }
114 |
115 | if (LOG.isInfoEnabled()) {
116 | final String hostInfo =
117 | requesterHostname == null
118 | ? "(no host locality info)"
119 | : "(on host '" + requesterHostname + "')";
120 | LOG.info("Subtask {} {} is requesting a multi jdbc source split", subtaskId, hostInfo);
121 | }
122 |
123 | if (!splitsQueue.isEmpty()) {
124 | MultiJdbcPartitionSplit split = splitsQueue.removeFirst();
125 | LOG.info(
126 | "assign split [{}] {} to task {}",
127 | split.getJdbcUrl(),
128 | split.getQuery(),
129 | subtaskId);
130 | context.assignSplit(split, subtaskId);
131 | } else {
132 | context.signalNoMoreSplits(subtaskId);
133 | LOG.info("No more splits available for subtask {}", subtaskId);
134 | }
135 | }
136 |
137 | /**
138 | * Handles a custom source event from the source reader.
139 | *
140 | *
This method has a default implementation that does nothing, because it is only required to
141 | * be implemented by some sources, which have a custom event protocol between reader and
142 | * enumerator. The common events for reader registration and split requests are not dispatched
143 | * to this method, but rather invoke the {@link #addReader(int)} and {@link
144 | * #handleSplitRequest(int, String)} methods.
145 | *
146 | * @param subtaskId the subtask id of the source reader who sent the source event.
147 | * @param sourceEvent the source event from the source reader.
148 | */
149 | @Override
150 | public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
151 | LOG.error("Received unrecognized event: {}", sourceEvent);
152 | }
153 |
154 | /**
155 | * Add a split back to the split enumerator. It will only happen when a {@link SourceReader}
156 | * fails and there are splits assigned to it after the last successful checkpoint.
157 | *
158 | * @param splits The split to add back to the enumerator for reassignment.
159 | * @param subtaskId The id of the subtask to which the returned splits belong.
160 | */
161 | @Override
162 | public void addSplitsBack(List splits, int subtaskId) {
163 | LOG.debug("File Source Enumerator adds splits back: {}", splits);
164 | for (MultiJdbcPartitionSplit split : splits) {
165 | splitsQueue.addLast(split);
166 | }
167 | }
168 |
169 | /**
170 | * Add a new source reader with the given subtask ID.
171 | *
172 | * @param subtaskId the subtask ID of the new source reader.
173 | */
174 | @Override
175 | public void addReader(int subtaskId) {
176 | // this source is purely lazy-pull-based, nothing to do upon registration
177 | }
178 |
179 | /**
180 | * Creates a snapshot of the state of this split enumerator, to be stored in a checkpoint.
181 | *
182 | * The snapshot should contain the latest state of the enumerator: It should assume that all
183 | * operations that happened before the snapshot have successfully completed. For example all
184 | * splits assigned to readers via {@link SplitEnumeratorContext#assignSplit(SourceSplit, int)}
185 | * and {@link SplitEnumeratorContext#assignSplits(SplitsAssignment)}) don't need to be included
186 | * in the snapshot anymore.
187 | *
188 | *
This method takes the ID of the checkpoint for which the state is snapshotted. Most
189 | * implementations should be able to ignore this parameter, because for the contents of the
190 | * snapshot, it doesn't matter for which checkpoint it gets created. This parameter can be
191 | * interesting for source connectors with external systems where those systems are themselves
192 | * aware of checkpoints; for example in cases where the enumerator notifies that system about a
193 | * specific checkpoint being triggered.
194 | *
195 | * @param checkpointId The ID of the checkpoint for which the snapshot is created.
196 | * @return an object containing the state of the split enumerator.
197 | * @throws Exception when the snapshot cannot be taken.
198 | */
199 | @Override
200 | public MultiJdbcSourceEnumState snapshotState(long checkpointId) throws Exception {
201 | return null;
202 | }
203 |
204 | /**
205 | * Called to close the enumerator, in case it holds on to any resources, like threads or network
206 | * connections.
207 | */
208 | @Override
209 | public void close() throws IOException {
210 | // no resources to close
211 | }
212 |
213 | private void generateSplit() {
214 | String[] jdbcUrls = options.getDbURL().split(";");
215 | for (String url : jdbcUrls) {
216 | LOG.info("process [{}] url : {}", jdbcUrlCounter.incrementAndGet(), url);
217 | try (Connection conn =
218 | options.getUsername().isPresent() && options.getPassword().isPresent()
219 | ? DriverManager.getConnection(
220 | url,
221 | options.getUsername().get(),
222 | options.getPassword().get())
223 | : DriverManager.getConnection(url);
224 | Statement stmt = conn.createStatement()) {
225 | // find all tables
226 | Map> allTables = findTables(conn);
227 | // gen sql query
228 | for (String db : allTables.keySet()) {
229 | if (!options.getDialect().dialectName().equalsIgnoreCase("derby")) {
230 | stmt.executeQuery(String.format(USE_DATABASE, db));
231 | }
232 | for (String table : allTables.get(db)) {
233 | generateQuery(stmt, table, url, db);
234 | }
235 | }
236 | } catch (Exception e) {
237 | String msg =
238 | String.format(
239 | "multi-jdbc enumerator generate split error: %s", e.getMessage());
240 | LOG.error(msg);
241 | LOG.error(Arrays.toString(e.getStackTrace()));
242 | throw new RuntimeException(e);
243 | }
244 | }
245 | }
246 |
247 | private void generateQuery(Statement stmt, String table, String url, String db)
248 | throws SQLException {
249 | // get max min
250 | ResultSet maxMinRes =
251 | stmt.executeQuery(
252 | String.format(
253 | SELECT_MAX_MIN,
254 | readOptions.getPartitionColumnName().get(),
255 | readOptions.getPartitionColumnName().get(),
256 | table));
257 | BigInteger max = new BigInteger("0");
258 | BigInteger min = new BigInteger("0");
259 | while (maxMinRes.next()) {
260 | // just return if there is no data in the table
261 | if (maxMinRes.getString(1) == null || maxMinRes.getString(2) == null) {
262 | return;
263 | }
264 | max = new BigInteger(maxMinRes.getString(1));
265 | min = new BigInteger(maxMinRes.getString(2));
266 | }
267 | BigInteger step;
268 | if (readOptions.getBatchSize() != null && !readOptions.getNumPartitions().isPresent()) {
269 | // get count
270 | ResultSet countRes = stmt.executeQuery(String.format(SELECT_COUNT, table));
271 | long count = 0;
272 | while (countRes.next()) {
273 | count = Long.parseLong(countRes.getString(1));
274 | }
275 | // 这里如果count比分片小会引起step 超大溢出的问题,所以要判断一下
276 | if (readOptions.getBatchSize() > count) {
277 | step = max.add(min.negate());
278 | } else {
279 | step =
280 | max.add(min.negate())
281 | .multiply(
282 | new BigInteger(String.valueOf(readOptions.getBatchSize())))
283 | .divide(new BigInteger(String.valueOf(count)));
284 | }
285 | } else if (readOptions.getBatchSize() == null
286 | && readOptions.getNumPartitions().isPresent()) {
287 | step =
288 | max.add(min.negate())
289 | .divide(
290 | new BigInteger(
291 | String.valueOf(
292 | String.valueOf(
293 | readOptions
294 | .getNumPartitions()
295 | .get()))));
296 | } else {
297 | throw new RuntimeException("neither batch size or partition num is null");
298 | }
299 | final JdbcDialect dialect = options.getDialect();
300 | while (min.compareTo(max) <= 0) {
301 | String query = getSelectFromStatement(db, table, columns, new String[0], dialect);
302 | query +=
303 | " WHERE "
304 | + dialect.quoteIdentifier(readOptions.getPartitionColumnName().get())
305 | + " BETWEEN %s AND %s";
306 | query = String.format(query, min, min.add(step));
307 | if (options.getUsername().isPresent() && options.getPassword().isPresent()) {
308 | splitsQueue.addLast(
309 | new MultiJdbcPartitionSplit(
310 | query,
311 | url,
312 | options.getUsername().get(),
313 | options.getPassword().get(),
314 | queryCounter.get()));
315 | } else {
316 | splitsQueue.addLast(
317 | new MultiJdbcPartitionSplit(query, url, null, null, queryCounter.get()));
318 | }
319 | min = min.add(step).add(new BigInteger("1"));
320 | queryCounter.incrementAndGet();
321 | }
322 | }
323 |
324 | private Map> findTables(Connection conn) {
325 | Pattern schema = Pattern.compile(options.getSchemaName());
326 | Pattern table = Pattern.compile(options.getTableName());
327 | Map> tables = new HashMap<>();
328 | try {
329 | DatabaseMetaData metaData = conn.getMetaData();
330 | ResultSet schemaResult = metaData.getCatalogs();
331 | if (options.getDialect().dialectName().equalsIgnoreCase("derby")) {
332 | schemaResult = metaData.getSchemas(null, null);
333 | }
334 | while (schemaResult.next()) {
335 | String schemaName = schemaResult.getString(1);
336 | Matcher m = schema.matcher(schemaName);
337 | if (m.matches()) {
338 | tables.put(schemaName, new ArrayList<>());
339 | LOG.info("total schema count : {}", totalSchemaCounter.incrementAndGet());
340 | }
341 | }
342 | for (String schemaName : tables.keySet()) {
343 | ResultSet tableResult = metaData.getTables(schemaName, null, null, null);
344 | if (options.getDialect().dialectName().equalsIgnoreCase("derby")) {
345 | tableResult = metaData.getTables(null, schemaName, null, null);
346 | }
347 | while (tableResult.next()) {
348 | String strTableName = tableResult.getString("TABLE_NAME");
349 | Matcher m = table.matcher(strTableName);
350 | if (m.matches()) {
351 | tables.get(schemaName).add(strTableName);
352 | LOG.info("total table counter : {}", totalTableCounter.incrementAndGet());
353 | }
354 | }
355 | }
356 | } catch (Exception e) {
357 | LOG.error(e.getMessage());
358 | LOG.error(Arrays.toString(e.getStackTrace()));
359 | throw new RuntimeException(e);
360 | }
361 | return tables;
362 | }
363 |
364 | @VisibleForTesting
365 | public Deque getSplitsQueue() {
366 | return splitsQueue;
367 | }
368 |
369 | /** Get select fields statement by condition fields. Default use SELECT. */
370 | String getSelectFromStatement(
371 | String schemaName, String tableName, String[] selectFields, String[] conditionFields,
372 | JdbcDialect jdbcDialect) {
373 | String selectExpressions =
374 | Arrays.stream(selectFields)
375 | .map(jdbcDialect::quoteIdentifier)
376 | .collect(Collectors.joining(", "));
377 | String fieldExpressions =
378 | Arrays.stream(conditionFields)
379 | .map(f -> format("%s = :%s", jdbcDialect.quoteIdentifier(f), f))
380 | .collect(Collectors.joining(" AND "));
381 | return "SELECT "
382 | + selectExpressions
383 | + " FROM "
384 | + jdbcDialect.quoteIdentifier(schemaName)
385 | + "."
386 | + jdbcDialect.quoteIdentifier(tableName)
387 | + (conditionFields.length > 0 ? " WHERE " + fieldExpressions : "");
388 | }
389 | }
390 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcInputFormatReader.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeInformation;
22 | import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
23 | import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
24 | import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
25 | import org.apache.flink.connector.jdbc.internal.converter.JdbcRowConverter;
26 | import org.apache.flink.connector.jdbc.table.JdbcRowDataInputFormat;
27 | import org.apache.flink.table.data.RowData;
28 | import org.apache.flink.util.Preconditions;
29 | import org.slf4j.Logger;
30 | import org.slf4j.LoggerFactory;
31 |
32 | import java.io.IOException;
33 | import java.sql.Connection;
34 | import java.sql.PreparedStatement;
35 | import java.sql.ResultSet;
36 | import java.sql.SQLException;
37 |
38 | /** multi jdbc input format reader. */
39 | public class MultiJdbcInputFormatReader implements InputFormatReader {
40 |
41 | private static final long serialVersionUID = 2L;
42 | private static final Logger LOG = LoggerFactory.getLogger(MultiJdbcInputFormatReader.class);
43 |
44 | private JdbcConnectionProvider connectionProvider;
45 | private int fetchSize;
46 | private Boolean autoCommit;
47 | private String queryTemplate;
48 | private int resultSetType;
49 | private int resultSetConcurrency;
50 | private JdbcRowConverter rowConverter;
51 | private TypeInformation rowDataTypeInfo;
52 |
53 | private transient PreparedStatement statement;
54 | private transient ResultSet resultSet;
55 | private transient boolean hasNext;
56 |
57 | private MultiJdbcInputFormatReader(
58 | JdbcConnectionProvider connectionProvider,
59 | int fetchSize,
60 | Boolean autoCommit,
61 | String queryTemplate,
62 | int resultSetType,
63 | int resultSetConcurrency,
64 | JdbcRowConverter rowConverter,
65 | TypeInformation rowDataTypeInfo) {
66 | this.connectionProvider = connectionProvider;
67 | this.fetchSize = fetchSize;
68 | this.autoCommit = autoCommit;
69 | this.queryTemplate = queryTemplate;
70 | this.resultSetType = resultSetType;
71 | this.resultSetConcurrency = resultSetConcurrency;
72 | this.rowConverter = rowConverter;
73 | this.rowDataTypeInfo = rowDataTypeInfo;
74 | }
75 |
76 | /**
77 | * Gets the data type (as a {@link TypeInformation}) produced by this function or input format.
78 | *
79 | * @return The data type produced by this function or input format.
80 | */
81 | @Override
82 | public TypeInformation getProducedType() {
83 | return rowDataTypeInfo;
84 | }
85 |
86 | @Override
87 | public void openInputFormat() {
88 | // called once per inputFormat (on open)
89 | try {
90 | Connection dbConn = connectionProvider.getOrEstablishConnection();
91 | // set autoCommit mode only if it was explicitly configured.
92 | // keep connection default otherwise.
93 | if (autoCommit != null) {
94 | dbConn.setAutoCommit(autoCommit);
95 | }
96 | statement = dbConn.prepareStatement(queryTemplate, resultSetType, resultSetConcurrency);
97 | if (fetchSize == Integer.MIN_VALUE || fetchSize > 0) {
98 | statement.setFetchSize(fetchSize);
99 | }
100 | resultSet = statement.executeQuery();
101 | hasNext = resultSet.next();
102 | } catch (SQLException se) {
103 | throw new IllegalArgumentException("open() failed." + se.getMessage(), se);
104 | } catch (ClassNotFoundException cnfe) {
105 | throw new IllegalArgumentException(
106 | "JDBC-Class not found. - " + cnfe.getMessage(), cnfe);
107 | }
108 | }
109 |
110 | @Override
111 | public void closeInputFormat() {
112 | // called once per inputFormat (on close)
113 | try {
114 | if (statement != null) {
115 | statement.close();
116 | }
117 | } catch (SQLException se) {
118 | LOG.info("Inputformat Statement couldn't be closed - " + se.getMessage());
119 | } finally {
120 | statement = null;
121 | }
122 |
123 | connectionProvider.closeConnection();
124 | }
125 |
126 | /**
127 | * Closes all resources used.
128 | *
129 | * @throws IOException Indicates that a resource could not be closed.
130 | */
131 | @Override
132 | public void close() throws IOException {
133 | if (resultSet == null) {
134 | return;
135 | }
136 | try {
137 | resultSet.close();
138 | } catch (SQLException se) {
139 | LOG.info("Inputformat ResultSet couldn't be closed - " + se.getMessage());
140 | }
141 | }
142 |
143 | /**
144 | * Checks whether all data has been read.
145 | *
146 | * @return boolean value indication whether all data has been read.
147 | * @throws IOException
148 | */
149 | @Override
150 | public boolean reachedEnd() throws IOException {
151 | return !hasNext;
152 | }
153 |
154 | /**
155 | * Stores the next resultSet row in a tuple.
156 | *
157 | * @return row containing next {@link RowData}
158 | * @throws IOException
159 | */
160 | @Override
161 | public RowData nextRecord() throws IOException {
162 | try {
163 | if (!hasNext) {
164 | return null;
165 | }
166 | RowData row = rowConverter.toInternal(resultSet);
167 | // update hasNext after we've read the record
168 | hasNext = resultSet.next();
169 | return row;
170 | } catch (SQLException se) {
171 | throw new IOException("Couldn't read data - " + se.getMessage(), se);
172 | } catch (NullPointerException npe) {
173 | throw new IOException("Couldn't access resultSet", npe);
174 | }
175 | }
176 |
177 | /** Builder for {@link JdbcRowDataInputFormat}. */
178 | public static class Builder {
179 | private JdbcConnectionOptions.JdbcConnectionOptionsBuilder connOptionsBuilder;
180 | private int fetchSize;
181 | private Boolean autoCommit;
182 | private String queryTemplate;
183 | private JdbcRowConverter rowConverter;
184 | private TypeInformation rowDataTypeInfo;
185 | private int resultSetType = ResultSet.TYPE_FORWARD_ONLY;
186 | private int resultSetConcurrency = ResultSet.CONCUR_READ_ONLY;
187 |
188 | public Builder() {
189 | this.connOptionsBuilder = new JdbcConnectionOptions.JdbcConnectionOptionsBuilder();
190 | }
191 |
192 | public Builder setDrivername(String drivername) {
193 | this.connOptionsBuilder.withDriverName(drivername);
194 | return this;
195 | }
196 |
197 | public Builder setDBUrl(String dbURL) {
198 | this.connOptionsBuilder.withUrl(dbURL);
199 | return this;
200 | }
201 |
202 | public Builder setUsername(String username) {
203 | this.connOptionsBuilder.withUsername(username);
204 | return this;
205 | }
206 |
207 | public Builder setPassword(String password) {
208 | this.connOptionsBuilder.withPassword(password);
209 | return this;
210 | }
211 |
212 | public Builder setQuery(String query) {
213 | this.queryTemplate = query;
214 | return this;
215 | }
216 |
217 | public Builder setRowDataTypeInfo(TypeInformation rowDataTypeInfo) {
218 | this.rowDataTypeInfo = rowDataTypeInfo;
219 | return this;
220 | }
221 |
222 | public Builder setRowConverter(JdbcRowConverter rowConverter) {
223 | this.rowConverter = rowConverter;
224 | return this;
225 | }
226 |
227 | public Builder setFetchSize(int fetchSize) {
228 | Preconditions.checkArgument(
229 | fetchSize == Integer.MIN_VALUE || fetchSize > 0,
230 | "Illegal value %s for fetchSize, has to be positive or Integer.MIN_VALUE.",
231 | fetchSize);
232 | this.fetchSize = fetchSize;
233 | return this;
234 | }
235 |
236 | public Builder setAutoCommit(boolean autoCommit) {
237 | this.autoCommit = autoCommit;
238 | return this;
239 | }
240 |
241 | public Builder setResultSetType(int resultSetType) {
242 | this.resultSetType = resultSetType;
243 | return this;
244 | }
245 |
246 | public Builder setResultSetConcurrency(int resultSetConcurrency) {
247 | this.resultSetConcurrency = resultSetConcurrency;
248 | return this;
249 | }
250 |
251 | public MultiJdbcInputFormatReader build() {
252 | if (this.queryTemplate == null) {
253 | throw new NullPointerException("No query supplied");
254 | }
255 | if (this.rowConverter == null) {
256 | throw new NullPointerException("No row converter supplied");
257 | }
258 | return new MultiJdbcInputFormatReader(
259 | new SimpleJdbcConnectionProvider(connOptionsBuilder.build()),
260 | this.fetchSize,
261 | this.autoCommit,
262 | this.queryTemplate,
263 | this.resultSetType,
264 | this.resultSetConcurrency,
265 | this.rowConverter,
266 | this.rowDataTypeInfo);
267 | }
268 | }
269 | }
270 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcPartitionSplit.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.api.connector.source.SourceSplit;
22 |
23 | import java.io.Serializable;
24 | import java.util.Objects;
25 |
26 | /** partition split for multi jdbc. */
27 | public class MultiJdbcPartitionSplit implements SourceSplit, Serializable {
28 |
29 | private static final long serialVersionUID = 1L;
30 |
31 | private final String query;
32 |
33 | private final String jdbcUrl;
34 |
35 | private final String username;
36 |
37 | private final String password;
38 |
39 | private final Integer uid;
40 |
41 | public MultiJdbcPartitionSplit(String query, String jdbcUrl, String username, String password, Integer uid) {
42 | this.query = query;
43 | this.jdbcUrl = jdbcUrl;
44 | this.username = username;
45 | this.password = password;
46 | this.uid = uid;
47 | }
48 |
49 | public String getQuery() {
50 | return query;
51 | }
52 |
53 | public String getJdbcUrl() {
54 | return jdbcUrl;
55 | }
56 |
57 | public String getUsername() {
58 | return username;
59 | }
60 |
61 | public String getPassword() {
62 | return password;
63 | }
64 |
65 | /**
66 | * Get the split id of this source split.
67 | *
68 | * @return id of this source split.
69 | */
70 | @Override
71 | public String splitId() {
72 | return uid.toString();
73 | }
74 |
75 | @Override
76 | public boolean equals(Object o) {
77 | if (this == o) {
78 | return true;
79 | }
80 | if (o == null || getClass() != o.getClass()) {
81 | return false;
82 | }
83 | MultiJdbcPartitionSplit split = (MultiJdbcPartitionSplit) o;
84 | return Objects.equals(query, split.query)
85 | && Objects.equals(jdbcUrl, split.jdbcUrl)
86 | && Objects.equals(username, split.username)
87 | && Objects.equals(password, split.password)
88 | && Objects.equals(uid, split.uid);
89 | }
90 |
91 | @Override
92 | public int hashCode() {
93 | return Objects.hash(query, jdbcUrl, username, password, uid);
94 | }
95 |
96 | @Override
97 | public String toString() {
98 | return "MultiJdbcPartitionSplit{"
99 | + "query='"
100 | + query
101 | + '\''
102 | + ", jdbcUrl='"
103 | + jdbcUrl
104 | + '\''
105 | + ", username='"
106 | + username
107 | + '\''
108 | + ", password='"
109 | + password
110 | + '\''
111 | + ", uid="
112 | + uid
113 | + '}';
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcReader.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeInformation;
22 | import org.apache.flink.api.connector.source.SourceReaderContext;
23 | import org.apache.flink.configuration.Configuration;
24 | import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
25 | import org.apache.flink.connector.file.src.util.RecordAndPosition;
26 | import org.apache.flink.table.data.RowData;
27 | import org.apache.flink.table.types.logical.RowType;
28 |
29 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcOptions;
30 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcReadOptions;
31 |
32 | import java.util.Map;
33 |
34 | /** multi jdbc reader. */
35 | public class MultiJdbcReader
36 | extends SingleThreadMultiplexSourceReaderBase<
37 | RecordAndPosition, RowData, MultiJdbcPartitionSplit, MultiJdbcSplitState> {
38 |
39 | public MultiJdbcReader(
40 | MultiJdbcOptions multiJdbcOptions,
41 | MultiJdbcReadOptions readOptions,
42 | RowType rowType,
43 | TypeInformation typeInformation,
44 | Configuration config,
45 | SourceReaderContext context) {
46 | super(
47 | () ->
48 | new MultiJdbcSplitReader(
49 | multiJdbcOptions, readOptions, rowType, typeInformation),
50 | new JdbcRecordEmitter(),
51 | config,
52 | context);
53 | }
54 |
55 | @Override
56 | public void start() {
57 | // we request a split only if we did not get splits during the checkpoint restore
58 | if (getNumberOfCurrentlyAssignedSplits() == 0) {
59 | context.sendSplitRequest();
60 | }
61 | }
62 |
63 | /**
64 | * When new splits are added to the reader. The initialize the state of the new splits.
65 | *
66 | * @param split a newly added split.
67 | */
68 | @Override
69 | protected MultiJdbcSplitState initializedState(MultiJdbcPartitionSplit split) {
70 | return new MultiJdbcSplitState(split);
71 | }
72 |
73 | /**
74 | * Convert a mutable SplitStateT to immutable SplitT.
75 | *
76 | * @param splitId
77 | * @param splitState splitState.
78 | * @return an immutable Split state.
79 | */
80 | @Override
81 | protected MultiJdbcPartitionSplit toSplitType(String splitId, MultiJdbcSplitState splitState) {
82 | return splitState.getSplit();
83 | }
84 |
85 | /**
86 | * Handles the finished splits to clean the state if needed.
87 | *
88 | * @param finishedSplitIds
89 | */
90 | @Override
91 | protected void onSplitFinished(Map finishedSplitIds) {
92 | context.sendSplitRequest();
93 | }
94 | }
95 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcSource.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeInformation;
22 | import org.apache.flink.api.connector.source.Boundedness;
23 | import org.apache.flink.api.connector.source.Source;
24 | import org.apache.flink.api.connector.source.SourceReader;
25 | import org.apache.flink.api.connector.source.SourceReaderContext;
26 | import org.apache.flink.api.connector.source.SplitEnumerator;
27 | import org.apache.flink.api.connector.source.SplitEnumeratorContext;
28 | import org.apache.flink.configuration.Configuration;
29 | import org.apache.flink.configuration.ReadableConfig;
30 | import org.apache.flink.core.io.SimpleVersionedSerializer;
31 | import org.apache.flink.table.catalog.ResolvedSchema;
32 | import org.apache.flink.table.data.RowData;
33 | import org.apache.flink.table.types.logical.RowType;
34 |
35 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcOptions;
36 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcReadOptions;
37 |
38 | /** multi jdbc source. */
39 | public class MultiJdbcSource
40 | implements Source {
41 |
42 | private final MultiJdbcOptions options;
43 | private final MultiJdbcReadOptions readOptions;
44 | private final RowType rowType;
45 | private final String[] columns;
46 | private final ReadableConfig config;
47 | private final TypeInformation typeInformation;
48 |
49 | public MultiJdbcSource(
50 | MultiJdbcOptions options,
51 | MultiJdbcReadOptions readOptions,
52 | ResolvedSchema schema,
53 | ReadableConfig config,
54 | TypeInformation typeInformation) {
55 | this.options = options;
56 | this.readOptions = readOptions;
57 | this.config = config;
58 | this.typeInformation = typeInformation;
59 | this.rowType = (RowType) schema.toPhysicalRowDataType().getLogicalType();
60 | this.columns = schema.getColumnNames().toArray(new String[0]);
61 | }
62 |
63 | /**
64 | * Get the boundedness of this source.
65 | *
66 | * @return the boundedness of this source.
67 | */
68 | @Override
69 | public Boundedness getBoundedness() {
70 | return Boundedness.BOUNDED;
71 | }
72 |
73 | /**
74 | * Creates a new reader to read data from the splits it gets assigned. The reader starts fresh
75 | * and does not have any state to resume.
76 | *
77 | * @param readerContext The {@link SourceReaderContext context} for the source reader.
78 | * @return A new SourceReader.
79 | * @throws Exception The implementor is free to forward all exceptions directly. Exceptions
80 | * thrown from this method cause task failure/recovery.
81 | */
82 | @Override
83 | public SourceReader createReader(
84 | SourceReaderContext readerContext) throws Exception {
85 | return new MultiJdbcReader(
86 | options,
87 | readOptions,
88 | rowType,
89 | typeInformation,
90 | (Configuration) config,
91 | readerContext);
92 | }
93 |
94 | /**
95 | * Creates a new SplitEnumerator for this source, starting a new input.
96 | *
97 | * @param enumContext The {@link SplitEnumeratorContext context} for the split enumerator.
98 | * @return A new SplitEnumerator.
99 | * @throws Exception The implementor is free to forward all exceptions directly. * Exceptions
100 | * thrown from this method cause JobManager failure/recovery.
101 | */
102 | @Override
103 | public SplitEnumerator createEnumerator(
104 | SplitEnumeratorContext enumContext) throws Exception {
105 | return new MultiJdbcEnumerator(options, readOptions, columns, enumContext);
106 | }
107 |
108 | /**
109 | * Restores an enumerator from a checkpoint.
110 | *
111 | * @param enumContext The {@link SplitEnumeratorContext context} for the restored split
112 | * enumerator.
113 | * @param checkpoint The checkpoint to restore the SplitEnumerator from.
114 | * @return A SplitEnumerator restored from the given checkpoint.
115 | * @throws Exception The implementor is free to forward all exceptions directly. * Exceptions
116 | * thrown from this method cause JobManager failure/recovery.
117 | */
118 | @Override
119 | public SplitEnumerator restoreEnumerator(
120 | SplitEnumeratorContext enumContext,
121 | MultiJdbcSourceEnumState checkpoint)
122 | throws Exception {
123 | // 恢复就是从头拉取
124 | return new MultiJdbcEnumerator(options, readOptions, columns, enumContext);
125 | }
126 |
127 | /**
128 | * Creates a serializer for the source splits. Splits are serialized when sending them from
129 | * enumerator to reader, and when checkpointing the reader's current state.
130 | *
131 | * @return The serializer for the split type.
132 | */
133 | @Override
134 | public SimpleVersionedSerializer getSplitSerializer() {
135 | return new MultiJdbcSplitSerializer();
136 | }
137 |
138 | /**
139 | * Creates the serializer for the {@link SplitEnumerator} checkpoint. The serializer is used for
140 | * the result of the {@link SplitEnumerator#snapshotState()} method.
141 | *
142 | * @return The serializer for the SplitEnumerator checkpoint.
143 | */
144 | @Override
145 | public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() {
146 | return new MultiJdbcSourceEnumStateSerializer();
147 | }
148 | }
149 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcSourceEnumState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | /** multi jdbc source enumerator state. */
22 | public class MultiJdbcSourceEnumState {}
23 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcSourceEnumStateSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.core.io.SimpleVersionedSerializer;
22 |
23 | import java.io.IOException;
24 |
25 | /** multi jdbc source enumerator state serializer. */
26 | public class MultiJdbcSourceEnumStateSerializer
27 | implements SimpleVersionedSerializer {
28 | private static final int CURRENT_VERSION = 0;
29 |
30 | @Override
31 | public int getVersion() {
32 | return CURRENT_VERSION;
33 | }
34 |
35 | @Override
36 | public byte[] serialize(MultiJdbcSourceEnumState obj) throws IOException {
37 | return new byte[0];
38 | }
39 |
40 | @Override
41 | public MultiJdbcSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
42 | return null;
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcSplitReader.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeInformation;
22 | import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
23 | import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
24 | import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
25 | import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
26 | import org.apache.flink.connector.file.src.util.RecordAndPosition;
27 | import org.apache.flink.table.data.RowData;
28 | import org.apache.flink.table.types.logical.RowType;
29 | import org.slf4j.Logger;
30 | import org.slf4j.LoggerFactory;
31 |
32 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcOptions;
33 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcReadOptions;
34 |
35 | import javax.annotation.Nullable;
36 | import java.io.IOException;
37 | import java.util.ArrayDeque;
38 | import java.util.Queue;
39 |
40 | /** split reader for multi jdbc. */
41 | public class MultiJdbcSplitReader
42 | implements SplitReader, MultiJdbcPartitionSplit> {
43 |
44 | private static final Logger LOG = LoggerFactory.getLogger(MultiJdbcSplitReader.class);
45 |
46 | private InputFormatReader inputFormatReader;
47 | private final MultiJdbcOptions options;
48 | private final MultiJdbcReadOptions readOptions;
49 | private final RowType rowType;
50 | private final TypeInformation typeInformation;
51 |
52 | @Nullable private String currentSplitId;
53 |
54 | private final Queue splits;
55 |
56 | public MultiJdbcSplitReader(
57 | MultiJdbcOptions multiJdbcOptions,
58 | MultiJdbcReadOptions readOptions,
59 | RowType rowType,
60 | TypeInformation typeInformation) {
61 | this.options = multiJdbcOptions;
62 | this.readOptions = readOptions;
63 | this.rowType = rowType;
64 | this.typeInformation = typeInformation;
65 | this.splits = new ArrayDeque<>();
66 | }
67 |
68 | /**
69 | * Fetch elements into the blocking queue for the given splits. The fetch call could be blocking
70 | * but it should get unblocked when {@link #wakeUp()} is invoked. In that case, the
71 | * implementation may either decide to return without throwing an exception, or it can just
72 | * throw an interrupted exception. In either case, this method should be reentrant, meaning that
73 | * the next fetch call should just resume from where the last fetch call was waken up or
74 | * interrupted.
75 | *
76 | * @return the Ids of the finished splits.
77 | * @throws IOException when encountered IO errors, such as deserialization failures.
78 | */
79 | @Override
80 | public RecordsWithSplitIds> fetch() throws IOException {
81 | if (inputFormatReader == null) {
82 | final MultiJdbcPartitionSplit nextSplit = splits.poll();
83 | if (nextSplit == null) {
84 | throw new IOException("Cannot fetch from another split - no split remaining");
85 | }
86 |
87 | currentSplitId = nextSplit.splitId();
88 |
89 | MultiJdbcInputFormatReader.Builder inputFormatReaderBuilder =
90 | new MultiJdbcInputFormatReader.Builder()
91 | .setAutoCommit(readOptions.getAutoCommit())
92 | .setDBUrl(nextSplit.getJdbcUrl())
93 | .setDrivername(options.getDriverName())
94 | .setQuery(nextSplit.getQuery())
95 | .setRowConverter(options.getDialect().getRowConverter(rowType))
96 | .setRowDataTypeInfo(typeInformation);
97 |
98 | if (readOptions.getFetchSize() != 0) {
99 | inputFormatReaderBuilder.setFetchSize(readOptions.getFetchSize());
100 | }
101 |
102 | if (options.getUsername().isPresent() && options.getPassword().isPresent()) {
103 | inputFormatReaderBuilder
104 | .setUsername(options.getUsername().get())
105 | .setPassword(options.getPassword().get());
106 | }
107 | inputFormatReader = inputFormatReaderBuilder.build();
108 | inputFormatReader.openInputFormat();
109 | }
110 |
111 | if (!inputFormatReader.reachedEnd()) {
112 | return JdbcRowDataRecord.forRecord(currentSplitId, inputFormatReader);
113 | } else {
114 | inputFormatReader.closeInputFormat();
115 | inputFormatReader.close();
116 | inputFormatReader = null;
117 | return JdbcRowDataRecord.finishSplit(currentSplitId);
118 | }
119 | }
120 |
121 | /**
122 | * Handle the split changes. This call should be non-blocking.
123 | *
124 | * @param splitsChanges the split changes that the SplitReader needs to handle.
125 | */
126 | @Override
127 | public void handleSplitsChanges(SplitsChange splitsChanges) {
128 | if (!(splitsChanges instanceof SplitsAddition)) {
129 | throw new UnsupportedOperationException(
130 | String.format(
131 | "The SplitChange type of %s is not supported.",
132 | splitsChanges.getClass()));
133 | }
134 |
135 | LOG.debug("Handling split change {}", splitsChanges);
136 | splits.addAll(splitsChanges.splits());
137 | }
138 |
139 | /** Wake up the split reader in case the fetcher thread is blocking in {@link #fetch()}. */
140 | @Override
141 | public void wakeUp() {}
142 |
143 | /**
144 | * Close the split reader.
145 | *
146 | * @throws Exception if closing the split reader failed.
147 | */
148 | @Override
149 | public void close() throws Exception {
150 | if (inputFormatReader != null) {
151 | inputFormatReader.closeInputFormat();
152 | inputFormatReader.close();
153 | }
154 | }
155 | }
156 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcSplitSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | import org.apache.flink.core.io.SimpleVersionedSerializer;
22 |
23 | import java.io.ByteArrayInputStream;
24 | import java.io.ByteArrayOutputStream;
25 | import java.io.DataInputStream;
26 | import java.io.DataOutputStream;
27 | import java.io.IOException;
28 |
29 | /** multi jdbc split serializer. */
30 | public class MultiJdbcSplitSerializer
31 | implements SimpleVersionedSerializer {
32 |
33 | private static final int CURRENT_VERSION = 0;
34 |
35 | @Override
36 | public int getVersion() {
37 | return CURRENT_VERSION;
38 | }
39 |
40 | @Override
41 | public byte[] serialize(MultiJdbcPartitionSplit split) throws IOException {
42 | try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
43 | DataOutputStream out = new DataOutputStream(baos)) {
44 | out.writeUTF(split.splitId());
45 | out.writeUTF(split.getQuery());
46 | out.writeUTF(split.getJdbcUrl());
47 | if (split.getUsername() != null && split.getPassword() != null) {
48 | out.writeUTF("password");
49 | out.writeUTF(split.getUsername());
50 | out.writeUTF(split.getPassword());
51 | } else {
52 | out.writeUTF("none");
53 | }
54 | out.flush();
55 | return baos.toByteArray();
56 | }
57 | }
58 |
59 | @Override
60 | public MultiJdbcPartitionSplit deserialize(int version, byte[] serialized) throws IOException {
61 | try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
62 | DataInputStream in = new DataInputStream(bais)) {
63 | String uid = in.readUTF();
64 | String query = in.readUTF();
65 | String jdbcUrl = in.readUTF();
66 | String security = in.readUTF();
67 | String username = null;
68 | String password = null;
69 | if (security.equals("password")) {
70 | username = in.readUTF();
71 | password = in.readUTF();
72 | }
73 | return new MultiJdbcPartitionSplit(query, jdbcUrl, username, password, Integer.parseInt(uid));
74 | }
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcSplitState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector.table;
20 |
21 | /** split state for multi jdbc. */
22 | public class MultiJdbcSplitState {
23 |
24 | private MultiJdbcPartitionSplit split;
25 |
26 | public MultiJdbcSplitState(MultiJdbcPartitionSplit split) {
27 | this.split = split;
28 | }
29 |
30 | public MultiJdbcPartitionSplit getSplit() {
31 | return split;
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | com.yuanfudao.multi.jdbc.connector.table.MultiJdbcDynamicTableFactory
17 |
--------------------------------------------------------------------------------
/src/test/java/com/yuanfudao/multi/jdbc/connector/MultiJdbcDynamicTableSourceITCase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.yuanfudao.multi.jdbc.connector;
20 |
21 | import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSource;
22 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
23 | import org.apache.flink.table.api.EnvironmentSettings;
24 | import org.apache.flink.table.api.TableEnvironment;
25 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
26 | import org.apache.flink.table.planner.runtime.utils.StreamTestSink;
27 | import org.apache.flink.test.util.AbstractTestBase;
28 | import org.apache.flink.types.Row;
29 | import org.apache.flink.util.CollectionUtil;
30 | import org.junit.After;
31 | import org.junit.Before;
32 | import org.junit.Test;
33 |
34 | import java.sql.Connection;
35 | import java.sql.DriverManager;
36 | import java.sql.SQLException;
37 | import java.sql.Statement;
38 | import java.util.Iterator;
39 | import java.util.List;
40 | import java.util.stream.Collectors;
41 | import java.util.stream.Stream;
42 |
43 | import static org.junit.Assert.assertEquals;
44 |
45 | /** ITCase for {@link JdbcDynamicTableSource}. */
46 | public class MultiJdbcDynamicTableSourceITCase extends AbstractTestBase {
47 |
48 | public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
49 | public static final String DB_URL_1 = "jdbc:derby:memory:test1";
50 | public static final String DB_URL_2 = "jdbc:derby:memory:test2";
51 | public static final String TABLE_NAME = "jdbc_source";
52 | public static final String INPUT_TABLE = "JDBCSOURCE.*";
53 | public static final String INPUT_DB = "APP";
54 | public static final String INPUT_TABLE_1 = "jdbcSource_1";
55 | public static final String INPUT_TABLE_2 = "jdbcSource_2";
56 | public static final String INPUT_TABLE_3 = "other_3";
57 |
58 | public static StreamExecutionEnvironment env;
59 | public static TableEnvironment tEnv;
60 |
61 | @Before
62 | public void before() throws ClassNotFoundException, SQLException {
63 | env = StreamExecutionEnvironment.getExecutionEnvironment();
64 | EnvironmentSettings envSettings =
65 | EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
66 | tEnv = StreamTableEnvironment.create(env, envSettings);
67 |
68 | System.setProperty(
69 | "derby.stream.error.field", MultiJdbcDynamicTableSourceITCase.class.getCanonicalName() + ".DEV_NULL");
70 | Class.forName(DRIVER_CLASS);
71 |
72 | initTable(DB_URL_1);
73 | initTable(DB_URL_2);
74 | }
75 |
76 | private void initTable(String url) throws ClassNotFoundException, SQLException {
77 | try (Connection conn = DriverManager.getConnection(url + ";create=true");
78 | Statement statement = conn.createStatement()) {
79 | statement.executeUpdate(
80 | "CREATE TABLE "
81 | + INPUT_TABLE_1
82 | + " (id BIGINT NOT NULL,"
83 | + "context VARCHAR(255))");
84 | statement.executeUpdate(
85 | "CREATE TABLE "
86 | + INPUT_TABLE_2
87 | + " ("
88 | + "id BIGINT NOT NULL,"
89 | + "context VARCHAR(255))");
90 | statement.executeUpdate(
91 | "CREATE TABLE "
92 | + INPUT_TABLE_3
93 | + " ("
94 | + "id BIGINT NOT NULL,"
95 | + "context_other VARCHAR(255))");
96 | statement.executeUpdate(
97 | "INSERT INTO " + INPUT_TABLE_1 + " VALUES (" + "1, 'a'),(2,'b'),(3,'c')");
98 | statement.executeUpdate(
99 | "INSERT INTO " + INPUT_TABLE_2 + " VALUES (" + "4, 'a1'),(5,'b1'),(6,'c1')");
100 | statement.executeUpdate(
101 | "INSERT INTO " + INPUT_TABLE_3 + " VALUES (" + "7, 'a2'),(8,'b2'),(9,'c2')");
102 | }
103 | }
104 |
105 | private void dropTable(String url) throws Exception {
106 | try (Connection conn = DriverManager.getConnection(url);
107 | Statement stat = conn.createStatement()) {
108 | stat.executeUpdate("DROP TABLE " + INPUT_TABLE_1);
109 | stat.executeUpdate("DROP TABLE " + INPUT_TABLE_2);
110 | stat.executeUpdate("DROP TABLE " + INPUT_TABLE_3);
111 | }
112 | }
113 |
114 | @After
115 | public void clearOutputTable() throws Exception {
116 | Class.forName(DRIVER_CLASS);
117 | dropTable(DB_URL_1);
118 | dropTable(DB_URL_2);
119 | StreamTestSink.clear();
120 | }
121 |
122 | @Test
123 | public void testJdbcSource() throws Exception {
124 | tEnv.executeSql(
125 | "CREATE TABLE "
126 | + TABLE_NAME
127 | + "("
128 | + "id BIGINT NOT NULL,"
129 | + "context VARCHAR(255)"
130 | + ") WITH ("
131 | + " 'connector'='multi-jdbc',"
132 | + " 'url'='"
133 | + String.format("%s;%s", DB_URL_1, DB_URL_2)
134 | + "',"
135 | + " 'table-name'='"
136 | + INPUT_TABLE
137 | + "',"
138 | + " 'schema-name'='"
139 | + INPUT_DB
140 | + "',"
141 | + "'scan.partition.column' = 'id',"
142 | + " 'scan.batch.size' = '2'"
143 | + ")");
144 |
145 | Iterator collected = tEnv.executeSql("SELECT * FROM " + TABLE_NAME).collect();
146 | List result =
147 | CollectionUtil.iteratorToList(collected).stream()
148 | .map(Row::toString)
149 | .sorted()
150 | .collect(Collectors.toList());
151 | List expected =
152 | Stream.of(
153 | "+I[1, a]",
154 | "+I[1, a]",
155 | "+I[2, b]",
156 | "+I[2, b]",
157 | "+I[3, c]",
158 | "+I[3, c]",
159 | "+I[4, a1]",
160 | "+I[4, a1]",
161 | "+I[5, b1]",
162 | "+I[5, b1]",
163 | "+I[6, c1]",
164 | "+I[6, c1]")
165 | .sorted()
166 | .collect(Collectors.toList());
167 | assertEquals(expected, result);
168 | }
169 | }
170 |
--------------------------------------------------------------------------------
/src/test/java/com/yuanfudao/multi/jdbc/connector/MultiSourceEnumeratorTests.java:
--------------------------------------------------------------------------------
1 | package com.yuanfudao.multi.jdbc.connector;
2 |
3 | import com.yuanfudao.multi.jdbc.connector.table.MultiJdbcEnumerator;
4 | import com.yuanfudao.multi.jdbc.connector.table.MultiJdbcPartitionSplit;
5 | import org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext;
6 | import org.apache.flink.table.api.DataTypes;
7 | import org.apache.flink.table.catalog.Column;
8 | import org.apache.flink.table.catalog.ResolvedSchema;
9 | import org.apache.flink.test.util.AbstractTestBase;
10 |
11 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcOptions;
12 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcReadOptions;
13 |
14 | import org.junit.After;
15 | import org.junit.Assert;
16 | import org.junit.Before;
17 | import org.junit.Test;
18 |
19 | import java.sql.Connection;
20 | import java.sql.DriverManager;
21 | import java.sql.SQLException;
22 | import java.sql.Statement;
23 | import java.util.ArrayDeque;
24 | import java.util.Deque;
25 |
26 | /** multi jdbc source enumerator logical test case. */
27 | public class MultiSourceEnumeratorTests extends AbstractTestBase {
28 |
29 | public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
30 | public static final String DB_URL_1 = "jdbc:derby:memory:test1";
31 | public static final String DB_URL_2 = "jdbc:derby:memory:test2";
32 | public static final String INPUT_TABLE_1 = "jdbcSource_1";
33 | public static final String INPUT_TABLE_2 = "jdbcSource_2";
34 | public static final String INPUT_TABLE_3 = "other_3";
35 |
36 | @Before
37 | public void before() throws ClassNotFoundException, SQLException {
38 | System.setProperty(
39 | "derby.stream.error.field", MultiSourceEnumeratorTests.class.getCanonicalName() + ".DEV_NULL");
40 | Class.forName(DRIVER_CLASS);
41 |
42 | initTable(DB_URL_1);
43 | initTable(DB_URL_2);
44 | }
45 |
46 | private void initTable(String url) throws ClassNotFoundException, SQLException {
47 | try (Connection conn = DriverManager.getConnection(url + ";create=true");
48 | Statement statement = conn.createStatement()) {
49 | statement.executeUpdate(
50 | "CREATE TABLE "
51 | + INPUT_TABLE_1
52 | + " (id BIGINT NOT NULL,"
53 | + "context VARCHAR(255))");
54 | statement.executeUpdate(
55 | "CREATE TABLE "
56 | + INPUT_TABLE_2
57 | + " ("
58 | + "id BIGINT NOT NULL,"
59 | + "context VARCHAR(255))");
60 | statement.executeUpdate(
61 | "CREATE TABLE "
62 | + INPUT_TABLE_3
63 | + " ("
64 | + "id BIGINT NOT NULL,"
65 | + "context_other VARCHAR(255))");
66 | statement.executeUpdate(
67 | "INSERT INTO " + INPUT_TABLE_1 + " VALUES (" + "1, 'a'),(2,'b'),(3,'c')");
68 | statement.executeUpdate(
69 | "INSERT INTO " + INPUT_TABLE_2 + " VALUES (" + "4, 'a1'),(5,'b1'),(6,'c1')");
70 | statement.executeUpdate(
71 | "INSERT INTO " + INPUT_TABLE_3 + " VALUES (" + "7, 'a2'),(8,'b2'),(9,'c2')");
72 | }
73 | }
74 |
75 | private void dropTable(String url) throws Exception {
76 | try (Connection conn = DriverManager.getConnection(url);
77 | Statement stat = conn.createStatement()) {
78 | stat.executeUpdate("DROP TABLE " + INPUT_TABLE_1);
79 | stat.executeUpdate("DROP TABLE " + INPUT_TABLE_2);
80 | stat.executeUpdate("DROP TABLE " + INPUT_TABLE_3);
81 | }
82 | }
83 |
84 | @After
85 | public void clearOutputTable() throws Exception {
86 | Class.forName(DRIVER_CLASS);
87 | dropTable(DB_URL_1);
88 | dropTable(DB_URL_2);
89 | }
90 |
91 | @Test
92 | public void testBatchSize() {
93 | final TestingSplitEnumeratorContext context =
94 | new TestingSplitEnumeratorContext<>(4);
95 | MultiJdbcOptions jdbcOptions =
96 | MultiJdbcOptions.builder()
97 | .setDBUrl("jdbc:derby:memory:test1;jdbc:derby:memory:test2")
98 | .setDriverName(DRIVER_CLASS)
99 | .setTableName("JDBCSOURCE.*")
100 | .setSchemaName("APP")
101 | .build();
102 |
103 | MultiJdbcReadOptions jdbcReadOptions =
104 | MultiJdbcReadOptions.builder().setPartitionColumnName("id").setBatchSize(2).build();
105 |
106 | MultiJdbcEnumerator multiJdbcEnumerator =
107 | new MultiJdbcEnumerator(jdbcOptions, jdbcReadOptions, genSchema(), context);
108 | multiJdbcEnumerator.start();
109 | Deque deque = multiJdbcEnumerator.getSplitsQueue();
110 | Deque except = new ArrayDeque<>();
111 | except.addLast(
112 | new MultiJdbcPartitionSplit(
113 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 1 AND 2",
114 | DB_URL_1,
115 | null,
116 | null,
117 | 0));
118 | except.addLast(
119 | new MultiJdbcPartitionSplit(
120 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 3 AND 4",
121 | DB_URL_1,
122 | null,
123 | null,
124 | 1));
125 | except.addLast(
126 | new MultiJdbcPartitionSplit(
127 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 4 AND 5",
128 | DB_URL_1,
129 | null,
130 | null,
131 | 2));
132 | except.addLast(
133 | new MultiJdbcPartitionSplit(
134 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 6 AND 7",
135 | DB_URL_1,
136 | null,
137 | null,
138 | 3));
139 |
140 | except.addLast(
141 | new MultiJdbcPartitionSplit(
142 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 1 AND 2",
143 | DB_URL_2,
144 | null,
145 | null,
146 | 4));
147 | except.addLast(
148 | new MultiJdbcPartitionSplit(
149 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 3 AND 4",
150 | DB_URL_2,
151 | null,
152 | null,
153 | 5));
154 | except.addLast(
155 | new MultiJdbcPartitionSplit(
156 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 4 AND 5",
157 | DB_URL_2,
158 | null,
159 | null,
160 | 6));
161 | except.addLast(
162 | new MultiJdbcPartitionSplit(
163 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 6 AND 7",
164 | DB_URL_2,
165 | null,
166 | null,
167 | 7));
168 | Assert.assertEquals(except.size(), deque.size());
169 | while (!except.isEmpty()) {
170 | Assert.assertEquals(except.removeLast(), deque.removeLast());
171 | }
172 | }
173 |
174 | @Test
175 | public void testPartitionNum() {
176 | final TestingSplitEnumeratorContext context =
177 | new TestingSplitEnumeratorContext<>(4);
178 | MultiJdbcOptions jdbcOptions =
179 | MultiJdbcOptions.builder()
180 | .setDBUrl("jdbc:derby:memory:test1;jdbc:derby:memory:test2")
181 | .setDriverName(DRIVER_CLASS)
182 | .setTableName("JDBCSOURCE.*")
183 | .setSchemaName("APP")
184 | .build();
185 |
186 | MultiJdbcReadOptions jdbcReadOptions =
187 | MultiJdbcReadOptions.builder()
188 | .setPartitionColumnName("id")
189 | .setNumPartitions(2)
190 | .build();
191 |
192 | MultiJdbcEnumerator multiJdbcEnumerator =
193 | new MultiJdbcEnumerator(jdbcOptions, jdbcReadOptions, genSchema(), context);
194 | multiJdbcEnumerator.start();
195 | Deque deque = multiJdbcEnumerator.getSplitsQueue();
196 | Deque except = new ArrayDeque<>();
197 | except.addLast(
198 | new MultiJdbcPartitionSplit(
199 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 1 AND 2",
200 | DB_URL_1,
201 | null,
202 | null,
203 | 0));
204 | except.addLast(
205 | new MultiJdbcPartitionSplit(
206 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 3 AND 4",
207 | DB_URL_1,
208 | null,
209 | null,
210 | 1));
211 | except.addLast(
212 | new MultiJdbcPartitionSplit(
213 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 4 AND 5",
214 | DB_URL_1,
215 | null,
216 | null,
217 | 2));
218 | except.addLast(
219 | new MultiJdbcPartitionSplit(
220 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 6 AND 7",
221 | DB_URL_1,
222 | null,
223 | null,
224 | 3));
225 |
226 | except.addLast(
227 | new MultiJdbcPartitionSplit(
228 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 1 AND 2",
229 | DB_URL_2,
230 | null,
231 | null,
232 | 4));
233 | except.addLast(
234 | new MultiJdbcPartitionSplit(
235 | "SELECT id, context FROM APP.JDBCSOURCE_1 WHERE id BETWEEN 3 AND 4",
236 | DB_URL_2,
237 | null,
238 | null,
239 | 5));
240 | except.addLast(
241 | new MultiJdbcPartitionSplit(
242 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 4 AND 5",
243 | DB_URL_2,
244 | null,
245 | null,
246 | 6));
247 | except.addLast(
248 | new MultiJdbcPartitionSplit(
249 | "SELECT id, context FROM APP.JDBCSOURCE_2 WHERE id BETWEEN 6 AND 7",
250 | DB_URL_2,
251 | null,
252 | null,
253 | 7));
254 | Assert.assertEquals(except.size(), deque.size());
255 | while (!except.isEmpty()) {
256 | Assert.assertEquals(except.removeLast(), deque.removeLast());
257 | }
258 | }
259 |
260 | private String[] genSchema() {
261 | return ResolvedSchema.of(
262 | Column.physical("id", DataTypes.BIGINT()),
263 | Column.physical("context", DataTypes.STRING()))
264 | .getColumnNames()
265 | .toArray(new String[0]);
266 | }
267 | }
268 |
--------------------------------------------------------------------------------