wuchong commented on a change in pull request #8923: [FLINK-13026][jdbc] Introduce JDBCLookupFunction URL: https://github.com/apache/flink/pull/8923#discussion_r300314109
########## File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java ########## @@ -0,0 +1,340 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.java.io.jdbc; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.types.Row; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.api.java.io.jdbc.JDBCTypeUtil.getFieldFromResultSet; +import static org.apache.flink.api.java.io.jdbc.JDBCTypeUtil.setFieldToStatement; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link TableFunction} to query fields from JDBC by keys. + * The query template like: + * <PRE> + * SELECT c, d, e, f from T where a = ? and b = ? + * </PRE> + * + * Support cache the result to avoid frequent accessing to remote databases. + * 1.The cacheMaxSize is -1 means not use cache. + * 2.For real-time data, you need to set the TTL of cache. + */ +public class JDBCLookupFunction extends TableFunction<Row> { + + private static final Logger LOG = LoggerFactory.getLogger(JDBCLookupFunction.class); + + static final int DEFAULT_MAX_RETRY_TIMES = 3; + + private final String query; + private final String drivername; + private final String dbURL; + private final String username; + private final String password; + private final TypeInformation[] keyTypes; + private final int[] keySqlTypes; + private final String[] fieldNames; + private final TypeInformation[] fieldTypes; + private final int[] outputSqlTypes; + private final long cacheMaxSize; + private final long cacheExpireMs; + private final int maxRetryTimes; + + private transient Connection dbConn; + private transient PreparedStatement statement; + private transient Cache<Row, List<Row>> cache; + + public JDBCLookupFunction( + String tableName, String username, String password, String drivername, String dbURL, + String[] fieldNames, TypeInformation[] fieldTypes, String[] keyNames, + String leftQuote, String rightQuote, long cacheMaxSize, long cacheExpireMs, int maxRetryTimes) { + this.drivername = drivername; + this.dbURL = dbURL; + this.username = username; + this.password = password; + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + List<String> nameList = Arrays.asList(fieldNames); + this.keyTypes = Arrays.stream(keyNames) + .map(s -> { + checkArgument(nameList.contains(s), + "keyName %s can't find in fieldNames %s.", s, nameList); + return fieldTypes[nameList.indexOf(s)]; + }) + .toArray(TypeInformation[]::new); + this.cacheMaxSize = cacheMaxSize; + this.cacheExpireMs = cacheExpireMs; + this.maxRetryTimes = maxRetryTimes; + this.keySqlTypes = Arrays.stream(keyTypes).mapToInt(JDBCTypeUtil::typeInformationToSqlType).toArray(); + this.outputSqlTypes = Arrays.stream(fieldTypes).mapToInt(JDBCTypeUtil::typeInformationToSqlType).toArray(); + + String quoteTableName = leftQuote + tableName + rightQuote; + String selectFields = StringUtils.join(Arrays.stream(fieldNames) + .map(name -> leftQuote + name + rightQuote) + .toArray(String[]::new), ","); + String conditions = StringUtils.join(Arrays.stream(keyNames) + .map(name -> leftQuote + name + rightQuote + " = ?") + .toArray(String[]::new), " AND "); + this.query = "SELECT " + selectFields + " FROM " + quoteTableName + " WHERE " + conditions; Review comment: We can add the lookup query and quote to `JDBCDialect` after https://github.com/apache/flink/pull/8867 is merged. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
