├── README.md ├── pom.xml └── src ├── main ├── java │ └── com │ │ └── yuanfudao │ │ └── multi │ │ └── jdbc │ │ └── connector │ │ ├── options │ │ ├── MultiJdbcOptions.java │ │ └── MultiJdbcReadOptions.java │ │ └── table │ │ ├── InputFormatReader.java │ │ ├── JdbcRecordEmitter.java │ │ ├── JdbcRowDataRecord.java │ │ ├── MultiJdbcDynamicTableFactory.java │ │ ├── MultiJdbcDynamicTableSource.java │ │ ├── MultiJdbcEnumerator.java │ │ ├── MultiJdbcInputFormatReader.java │ │ ├── MultiJdbcPartitionSplit.java │ │ ├── MultiJdbcReader.java │ │ ├── MultiJdbcSource.java │ │ ├── MultiJdbcSourceEnumState.java │ │ ├── MultiJdbcSourceEnumStateSerializer.java │ │ ├── MultiJdbcSplitReader.java │ │ ├── MultiJdbcSplitSerializer.java │ │ └── MultiJdbcSplitState.java └── resources │ └── META-INF │ └── services │ └── org.apache.flink.table.factories.Factory └── test └── java └── com └── yuanfudao └── multi └── jdbc └── connector ├── MultiJdbcDynamicTableSourceITCase.java └── MultiSourceEnumeratorTests.java /README.md: -------------------------------------------------------------------------------- 1 | # FLINK MULTI JDBC CONNECTOR 2 | 3 | 提供分库分表的jdbc链接方法,目前只提供批模式。 4 | 5 | ## 依赖 6 | flink 1.13.2 7 | 8 | ## 使用方式 9 | ### 参数 10 | |参数名称|是否必填|说明| 11 | |-------|-------|----| 12 | |connector|是|填写multi-jdbc| 13 | |url |是 |jdbc的链接串,用分号分割。jdbc:mysql://${connection1};jdbc:mysql://${connection2} | 14 | |table-name |是 |可用正则匹配 | 15 | |schema-name |是 |可用正则匹配 | 16 | |username |否 |用户名 | 17 | |password |否 |密码 | 18 | |driver |否 | | 19 | |scan.partition.column |是 |分片的字段,最好是自增id主键 | 20 | |scan.batch.size |否 |scan.batch.size 和 scan.partition.num必填写一个。每个batch的大小,每个表会多一个select count(1)的查询去获取表的数据量进行计算 | 21 | |scan.partition.num |否 |scan.batch.size 和 scan.partition.num必填写一个。每张表分多少个片。 | 22 | 23 | ### 例子 24 | ```sql 25 | CREATE TABLE test_multi_jdbc( 26 | `id` BIGINT COMMENT '', 27 | `dbctime` TIMESTAMP(3) COMMENT '', 28 | `dbutime` TIMESTAMP(3) COMMENT '', 29 | `extension` STRING COMMENT '') COMMENT 'null' 30 | WITH ( 31 | 'connector' = 'multi-jdbc', 32 | 'table-name' = 'test_table.*', 33 | 'schema-name' = 'test_database.*', 34 | 'username' = '${username}', 35 | 'password' = '${password}', 36 | 'scan.partition.column' = 'id', 37 | 'scan.batch.size' = '100000', 38 | 'url' = 'jdbc:mysql://${connection1};jdbc:mysql://${connection2}' 39 | ) 40 | ``` 41 | 42 | ### 设计 43 | 基于flip-27设计的flink接口,实现的分库分表jdbc的链接器。 44 | enumerator(在jobmanager内)负责 45 | 1. 把符合条件的库名找出来。 46 | 2. 把符合表名称的表明拿出来。 47 | 3. 取这张表的最大值以及最小值。 48 | 4. 获取一个批次的步长。 49 | 1. 若填写了batch size。则通过select count 获取表行数计算,在使用 批次大小/总行数 * (max id - min id)计算出步长。 50 | 2. 拖填写了partition num。 则使用 (max id - min id) / 分片数量 计算出步长。 51 | 5. 根据步长来生成对应的sql(select * from xxxx where column between xxxx and xxxx) -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.yuanfudao 8 | FlinkMultiJdbcConnector 9 | 1.0-SNAPSHOT 10 | 11 | jar 12 | 13 | 14 | 8 15 | 8 16 | 1.13.2 17 | 2.11 18 | 19 | 20 | 21 | 22 | org.apache.flink 23 | flink-connector-jdbc_${scala.binary.version} 24 | ${flink.version} 25 | provided 26 | 27 | 28 | 29 | 30 | org.apache.flink 31 | flink-table-api-java-bridge_${scala.binary.version} 32 | ${flink.version} 33 | provided 34 | true 35 | 36 | 37 | 38 | 39 | 40 | org.apache.flink 41 | flink-test-utils_${scala.binary.version} 42 | ${flink.version} 43 | test 44 | 45 | 46 | 47 | org.apache.flink 48 | flink-connector-test-utils 49 | ${flink.version} 50 | test 51 | 52 | 53 | 54 | org.apache.flink 55 | flink-table-common 56 | ${flink.version} 57 | test-jar 58 | test 59 | 60 | 61 | 62 | org.apache.flink 63 | flink-streaming-java_${scala.binary.version} 64 | ${flink.version} 65 | test-jar 66 | test 67 | 68 | 69 | 70 | org.apache.flink 71 | flink-core 72 | ${flink.version} 73 | test 74 | test-jar 75 | 76 | 77 | 78 | 79 | org.apache.flink 80 | flink-table-planner_${scala.binary.version} 81 | ${flink.version} 82 | test 83 | 84 | 85 | 86 | org.apache.flink 87 | flink-table-planner-blink_${scala.binary.version} 88 | ${flink.version} 89 | test-jar 90 | test 91 | 92 | 93 | 94 | org.apache.flink 95 | flink-table-planner-blink_${scala.binary.version} 96 | ${flink.version} 97 | test 98 | 99 | 100 | 101 | 102 | 103 | org.apache.derby 104 | derby 105 | 10.14.2.0 106 | test 107 | 108 | 109 | 110 | -------------------------------------------------------------------------------- /src/main/java/com/yuanfudao/multi/jdbc/connector/options/MultiJdbcOptions.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.options; 20 | 21 | import org.apache.flink.connector.jdbc.JdbcConnectionOptions; 22 | import org.apache.flink.connector.jdbc.dialect.JdbcDialect; 23 | import org.apache.flink.connector.jdbc.dialect.JdbcDialects; 24 | import org.apache.flink.connector.jdbc.internal.options.JdbcOptions; 25 | 26 | import java.util.Objects; 27 | import java.util.Optional; 28 | 29 | import static org.apache.flink.util.Preconditions.checkNotNull; 30 | 31 | /** multi jdbc options. */ 32 | public class MultiJdbcOptions extends JdbcConnectionOptions { 33 | private String schemaName; 34 | private String tableName; 35 | private JdbcDialect dialect; 36 | 37 | public MultiJdbcOptions( 38 | String schemaName, 39 | String dbURL, 40 | String tableName, 41 | String driverName, 42 | String username, 43 | String password, 44 | JdbcDialect dialect, 45 | int connectionCheckTimeoutSeconds) { 46 | super(dbURL, driverName, username, password, connectionCheckTimeoutSeconds); 47 | this.tableName = tableName; 48 | this.schemaName = schemaName; 49 | this.dialect = dialect; 50 | } 51 | 52 | public String getSchemaName() { 53 | return schemaName; 54 | } 55 | 56 | public String getTableName() { 57 | return tableName; 58 | } 59 | 60 | public JdbcDialect getDialect() { 61 | return dialect; 62 | } 63 | 64 | public static Builder builder() { 65 | return new Builder(); 66 | } 67 | 68 | @Override 69 | public boolean equals(Object o) { 70 | if (this == o) { 71 | return true; 72 | } 73 | if (o == null || getClass() != o.getClass()) { 74 | return false; 75 | } 76 | MultiJdbcOptions that = (MultiJdbcOptions) o; 77 | return Objects.equals(schemaName, that.schemaName) 78 | && Objects.equals(tableName, that.tableName); 79 | } 80 | 81 | @Override 82 | public int hashCode() { 83 | return Objects.hash(schemaName, tableName); 84 | } 85 | 86 | /** Builder of {@link JdbcOptions}. */ 87 | public static class Builder { 88 | private String dbURL; 89 | private String schemaName; 90 | private String tableName; 91 | private String driverName; 92 | private String username; 93 | private JdbcDialect dialect; 94 | private String password; 95 | private int connectionCheckTimeoutSeconds = 60; 96 | 97 | /** required, table name. */ 98 | public Builder setTableName(String tableName) { 99 | this.tableName = tableName; 100 | return this; 101 | } 102 | 103 | /** optional, user name. */ 104 | public Builder setUsername(String username) { 105 | this.username = username; 106 | return this; 107 | } 108 | 109 | /** optional, password. */ 110 | public Builder setPassword(String password) { 111 | this.password = password; 112 | return this; 113 | } 114 | 115 | /** optional, connectionCheckTimeoutSeconds. */ 116 | public Builder setConnectionCheckTimeoutSeconds(int connectionCheckTimeoutSeconds) { 117 | this.connectionCheckTimeoutSeconds = connectionCheckTimeoutSeconds; 118 | return this; 119 | } 120 | 121 | /** 122 | * optional, driver name, dialect has a default driver name, See {@link 123 | * JdbcDialect#defaultDriverName}. 124 | */ 125 | public Builder setDriverName(String driverName) { 126 | this.driverName = driverName; 127 | return this; 128 | } 129 | 130 | /** required, JDBC DB url. */ 131 | public Builder setDBUrl(String dbURL) { 132 | this.dbURL = dbURL; 133 | return this; 134 | } 135 | 136 | /** 137 | * optional, Handle the SQL dialect of jdbc driver. If not set, it will be infer by {@link 138 | * JdbcDialects#get} from DB url. 139 | */ 140 | public Builder setDialect(JdbcDialect dialect) { 141 | this.dialect = dialect; 142 | return this; 143 | } 144 | 145 | public Builder setSchemaName(String schemaName) { 146 | this.schemaName = schemaName; 147 | return this; 148 | } 149 | 150 | public MultiJdbcOptions build() { 151 | checkNotNull(dbURL, "No dbURL supplied."); 152 | checkNotNull(tableName, "No tableName supplied."); 153 | if (this.dialect == null) { 154 | Optional optional = JdbcDialects.get(dbURL); 155 | this.dialect = 156 | optional.orElseGet( 157 | () -> { 158 | throw new NullPointerException( 159 | "Unknown dbURL,can not find proper dialect."); 160 | }); 161 | } 162 | if (this.driverName == null) { 163 | Optional optional = dialect.defaultDriverName(); 164 | this.driverName = 165 | optional.orElseGet( 166 | () -> { 167 | throw new NullPointerException("No driverName supplied."); 168 | }); 169 | } 170 | 171 | return new MultiJdbcOptions( 172 | schemaName, 173 | dbURL, 174 | tableName, 175 | driverName, 176 | username, 177 | password, 178 | dialect, 179 | connectionCheckTimeoutSeconds); 180 | } 181 | } 182 | } 183 | -------------------------------------------------------------------------------- /src/main/java/com/yuanfudao/multi/jdbc/connector/options/MultiJdbcReadOptions.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.options; 20 | 21 | import java.io.Serializable; 22 | import java.util.Objects; 23 | import java.util.Optional; 24 | 25 | /** Options for the JDBC scan. */ 26 | public class MultiJdbcReadOptions implements Serializable { 27 | 28 | private final String query; 29 | private final String partitionColumnName; 30 | private final Long partitionLowerBound; 31 | private final Long partitionUpperBound; 32 | private final Integer numPartitions; 33 | private final Integer batchSize; 34 | 35 | private final int fetchSize; 36 | private final boolean autoCommit; 37 | 38 | private MultiJdbcReadOptions( 39 | String query, 40 | String partitionColumnName, 41 | Long partitionLowerBound, 42 | Long partitionUpperBound, 43 | Integer numPartitions, 44 | Integer batchSize, 45 | int fetchSize, 46 | boolean autoCommit) { 47 | this.query = query; 48 | this.partitionColumnName = partitionColumnName; 49 | this.partitionLowerBound = partitionLowerBound; 50 | this.partitionUpperBound = partitionUpperBound; 51 | this.numPartitions = numPartitions; 52 | this.batchSize = batchSize; 53 | 54 | this.fetchSize = fetchSize; 55 | this.autoCommit = autoCommit; 56 | } 57 | 58 | public Integer getBatchSize() { 59 | return batchSize; 60 | } 61 | 62 | public Optional getQuery() { 63 | return Optional.ofNullable(query); 64 | } 65 | 66 | public Optional getPartitionColumnName() { 67 | return Optional.ofNullable(partitionColumnName); 68 | } 69 | 70 | public Optional getPartitionLowerBound() { 71 | return Optional.ofNullable(partitionLowerBound); 72 | } 73 | 74 | public Optional getPartitionUpperBound() { 75 | return Optional.ofNullable(partitionUpperBound); 76 | } 77 | 78 | public Optional getNumPartitions() { 79 | return Optional.ofNullable(numPartitions); 80 | } 81 | 82 | public int getFetchSize() { 83 | return fetchSize; 84 | } 85 | 86 | public boolean getAutoCommit() { 87 | return autoCommit; 88 | } 89 | 90 | public static Builder builder() { 91 | return new Builder(); 92 | } 93 | 94 | @Override 95 | public boolean equals(Object o) { 96 | if (o instanceof MultiJdbcReadOptions) { 97 | MultiJdbcReadOptions options = (MultiJdbcReadOptions) o; 98 | return Objects.equals(query, options.query) 99 | && Objects.equals(partitionColumnName, options.partitionColumnName) 100 | && Objects.equals(partitionLowerBound, options.partitionLowerBound) 101 | && Objects.equals(partitionUpperBound, options.partitionUpperBound) 102 | && Objects.equals(numPartitions, options.numPartitions) 103 | && Objects.equals(batchSize, options.batchSize) 104 | && Objects.equals(fetchSize, options.fetchSize) 105 | && Objects.equals(autoCommit, options.autoCommit); 106 | } else { 107 | return false; 108 | } 109 | } 110 | 111 | /** Builder of {@link MultiJdbcReadOptions}. */ 112 | public static class Builder { 113 | protected String query; 114 | protected String partitionColumnName; 115 | protected Long partitionLowerBound; 116 | protected Long partitionUpperBound; 117 | protected Integer numPartitions; 118 | protected Integer batchSize; 119 | 120 | protected int fetchSize = 0; 121 | protected boolean autoCommit = false; 122 | 123 | /** optional, SQL query statement for this JDBC source. */ 124 | public Builder setQuery(String query) { 125 | this.query = query; 126 | return this; 127 | } 128 | 129 | /** optional, name of the column used for partitioning the input. */ 130 | public Builder setPartitionColumnName(String partitionColumnName) { 131 | this.partitionColumnName = partitionColumnName; 132 | return this; 133 | } 134 | 135 | /** optional, the smallest value of the first partition. */ 136 | public Builder setPartitionLowerBound(long partitionLowerBound) { 137 | this.partitionLowerBound = partitionLowerBound; 138 | return this; 139 | } 140 | 141 | /** optional, the largest value of the last partition. */ 142 | public Builder setPartitionUpperBound(long partitionUpperBound) { 143 | this.partitionUpperBound = partitionUpperBound; 144 | return this; 145 | } 146 | 147 | /** 148 | * optional, the maximum number of partitions that can be used for parallelism in table 149 | * reading. 150 | */ 151 | public Builder setNumPartitions(int numPartitions) { 152 | this.numPartitions = numPartitions; 153 | return this; 154 | } 155 | 156 | /** 157 | * optional, the number of rows to one batch. default value is 0, according to the jdbc api, 158 | * 0 means that batchSize hint will be ignored. 159 | */ 160 | public Builder setBatchSize(int batchSize) { 161 | this.batchSize = batchSize; 162 | return this; 163 | } 164 | 165 | /** 166 | * optional, the number of rows to fetch per round trip. default value is 0, according to 167 | * the jdbc api, 0 means that fetchSize hint will be ignored. 168 | */ 169 | public Builder setFetchSize(int fetchSize) { 170 | this.fetchSize = fetchSize; 171 | return this; 172 | } 173 | 174 | /** optional, whether to set auto commit on the JDBC driver. */ 175 | public Builder setAutoCommit(boolean autoCommit) { 176 | this.autoCommit = autoCommit; 177 | return this; 178 | } 179 | 180 | public MultiJdbcReadOptions build() { 181 | return new MultiJdbcReadOptions( 182 | query, 183 | partitionColumnName, 184 | partitionLowerBound, 185 | partitionUpperBound, 186 | numPartitions, 187 | batchSize, 188 | fetchSize, 189 | autoCommit); 190 | } 191 | } 192 | } 193 | -------------------------------------------------------------------------------- /src/main/java/com/yuanfudao/multi/jdbc/connector/table/InputFormatReader.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.java.typeutils.ResultTypeQueryable; 22 | import org.apache.flink.table.data.RowData; 23 | 24 | import java.io.IOException; 25 | import java.io.Serializable; 26 | 27 | /** input format reader. */ 28 | public interface InputFormatReader extends Serializable, ResultTypeQueryable { 29 | 30 | void openInputFormat(); 31 | 32 | void closeInputFormat(); 33 | 34 | boolean reachedEnd() throws IOException; 35 | 36 | RowData nextRecord() throws IOException; 37 | 38 | void close() throws IOException; 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/yuanfudao/multi/jdbc/connector/table/JdbcRecordEmitter.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.SourceOutput; 22 | import org.apache.flink.connector.base.source.reader.RecordEmitter; 23 | import org.apache.flink.connector.file.src.util.RecordAndPosition; 24 | import org.apache.flink.table.data.RowData; 25 | 26 | /** jdbc record emitter. */ 27 | public class JdbcRecordEmitter 28 | implements RecordEmitter, RowData, MultiJdbcSplitState> { 29 | /** 30 | * Process and emit the records to the {@link SourceOutput}. A few recommendations to the 31 | * implementation are following: 32 | * 33 | *
    34 | *
  • The method maybe interrupted in the middle. In that case, the same set of records will 35 | * be passed to the record emitter again later. The implementation needs to make sure it 36 | * reades 37 | *
  • 38 | *
39 | * 40 | * @param element The intermediate element read by the SplitReader. 41 | * @param output The output to which the final records are emit to. 42 | * @param splitState The state of the split. 43 | */ 44 | @Override 45 | public void emitRecord( 46 | RecordAndPosition element, 47 | SourceOutput output, 48 | MultiJdbcSplitState splitState) 49 | throws Exception { 50 | output.collect(element.getRecord()); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/yuanfudao/multi/jdbc/connector/table/JdbcRowDataRecord.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.connector.base.source.reader.RecordsWithSplitIds; 22 | import org.apache.flink.connector.file.src.util.RecordAndPosition; 23 | import org.apache.flink.table.data.RowData; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | import javax.annotation.Nullable; 28 | import java.util.Arrays; 29 | import java.util.Collections; 30 | import java.util.Set; 31 | 32 | /** jdbc row data record for multi jdbc. */ 33 | public class JdbcRowDataRecord implements RecordsWithSplitIds> { 34 | 35 | private static final Logger LOG = LoggerFactory.getLogger(JdbcRowDataRecord.class); 36 | 37 | @Nullable private String splitId; 38 | 39 | private final Set finishedSplits; 40 | 41 | private final InputFormatReader inputFormatReader; 42 | 43 | private InputFormatReader currentReader; 44 | 45 | long count = 0; 46 | 47 | public JdbcRowDataRecord( 48 | @Nullable String splitId, 49 | Set finishedSplits, 50 | InputFormatReader inputFormatReader) { 51 | this.splitId = splitId; 52 | this.finishedSplits = finishedSplits; 53 | this.inputFormatReader = inputFormatReader; 54 | } 55 | 56 | /** 57 | * Moves to the next split. This method is also called initially to move to the first split. 58 | * Returns null, if no splits are left. 59 | */ 60 | @Nullable 61 | @Override 62 | public String nextSplit() { 63 | // move the split one (from current value to null) 64 | final String nextSplit = this.splitId; 65 | this.splitId = null; 66 | count = 0; 67 | 68 | // move the iterator, from null to value (if first move) or to null (if second move) 69 | this.currentReader = nextSplit != null ? this.inputFormatReader : null; 70 | 71 | return nextSplit; 72 | } 73 | 74 | /** 75 | * Gets the next record from the current split. Returns null if no more records are left in this 76 | * split. 77 | */ 78 | @Nullable 79 | @Override 80 | public RecordAndPosition nextRecordFromSplit() { 81 | try { 82 | if (!currentReader.reachedEnd()) { 83 | return new RecordAndPosition<>(currentReader.nextRecord(), count++, 0); 84 | } 85 | } catch (Exception e) { 86 | LOG.error(e.getMessage()); 87 | LOG.error(Arrays.toString(e.getStackTrace())); 88 | throw new RuntimeException(e); 89 | } 90 | return null; 91 | } 92 | 93 | /** 94 | * Get the finished splits. 95 | * 96 | * @return the finished splits after this RecordsWithSplitIds is returned. 97 | */ 98 | @Override 99 | public Set finishedSplits() { 100 | return finishedSplits; 101 | } 102 | 103 | public static JdbcRowDataRecord forRecord( 104 | final String splitId, final InputFormatReader inputFormatReader) { 105 | return new JdbcRowDataRecord(splitId, Collections.emptySet(), inputFormatReader); 106 | } 107 | 108 | public static JdbcRowDataRecord finishSplit(final String splitId) { 109 | return new JdbcRowDataRecord(null, Collections.singleton(splitId), null); 110 | } 111 | } 112 | -------------------------------------------------------------------------------- /src/main/java/com/yuanfudao/multi/jdbc/connector/table/MultiJdbcDynamicTableFactory.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.configuration.ConfigOption; 22 | import org.apache.flink.configuration.ConfigOptions; 23 | import org.apache.flink.configuration.ReadableConfig; 24 | import org.apache.flink.connector.jdbc.dialect.JdbcDialects; 25 | import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSource; 26 | import org.apache.flink.table.catalog.CatalogTable; 27 | import org.apache.flink.table.connector.source.DynamicTableSource; 28 | import org.apache.flink.table.factories.DynamicTableSourceFactory; 29 | import org.apache.flink.table.factories.Factory; 30 | import org.apache.flink.table.factories.FactoryUtil; 31 | 32 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcOptions; 33 | import com.yuanfudao.multi.jdbc.connector.options.MultiJdbcReadOptions; 34 | 35 | import java.time.Duration; 36 | import java.util.HashSet; 37 | import java.util.Optional; 38 | import java.util.Set; 39 | 40 | /** Factory for creating configured instances of {@link JdbcDynamicTableSource}. */ 41 | public class MultiJdbcDynamicTableFactory implements DynamicTableSourceFactory { 42 | 43 | public static final String IDENTIFIER = "multi-jdbc"; 44 | public static final ConfigOption URL = 45 | ConfigOptions.key("url") 46 | .stringType() 47 | .noDefaultValue() 48 | .withDescription("The JDBC database URL."); 49 | public static final ConfigOption TABLE_NAME = 50 | ConfigOptions.key("table-name") 51 | .stringType() 52 | .noDefaultValue() 53 | .withDescription("The JDBC table name(support regex)."); 54 | 55 | public static final ConfigOption SCHEMA_NAME = 56 | ConfigOptions.key("schema-name") 57 | .stringType() 58 | .noDefaultValue() 59 | .withDescription("The JDBC schema name(support regex)."); 60 | public static final ConfigOption USERNAME = 61 | ConfigOptions.key("username") 62 | .stringType() 63 | .noDefaultValue() 64 | .withDescription("The JDBC user name."); 65 | public static final ConfigOption PASSWORD = 66 | ConfigOptions.key("password") 67 | .stringType() 68 | .noDefaultValue() 69 | .withDescription("The JDBC password."); 70 | private static final ConfigOption DRIVER = 71 | ConfigOptions.key("driver") 72 | .stringType() 73 | .noDefaultValue() 74 | .withDescription( 75 | "The class name of the JDBC driver to use to connect to this URL. " 76 | + "If not set, it will automatically be derived from the URL."); 77 | public static final ConfigOption MAX_RETRY_TIMEOUT = 78 | ConfigOptions.key("connection.max-retry-timeout") 79 | .durationType() 80 | .defaultValue(Duration.ofSeconds(60)) 81 | .withDescription("Maximum timeout between retries."); 82 | 83 | private static final ConfigOption SCAN_PARTITION_COLUMN = 84 | ConfigOptions.key("scan.partition.column") 85 | .stringType() 86 | .noDefaultValue() 87 | .withDescription("The column name used for partitioning the input."); 88 | private static final ConfigOption SCAN_BATCH_SIZE = 89 | ConfigOptions.key("scan.batch.size") 90 | .intType() 91 | .noDefaultValue() 92 | .withDescription("The number of partitions."); 93 | private static final ConfigOption 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 | --------------------------------------------------------------------------------