codope commented on a change in pull request #2915: URL: https://github.com/apache/hudi/pull/2915#discussion_r633690635
########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java ########## @@ -0,0 +1,326 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.SqlQueryBuilder; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.DataFrameReader; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.storage.StorageLevel; +import org.jetbrains.annotations.NotNull; + +import java.net.URI; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * Reads data from RDBMS data sources. + */ + +public class JdbcSource extends RowSource { + + private static final Logger LOG = LogManager.getLogger(JdbcSource.class); + private static final List<String> DB_LIMIT_CLAUSE = Arrays.asList("mysql", "postgresql", "h2"); + private static final String URI_JDBC_PREFIX = "jdbc:"; + + public JdbcSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + } + + /** + * Validates all user properties and prepares the {@link DataFrameReader} to read from RDBMS. + * + * @param session The {@link SparkSession}. + * @param properties The JDBC connection properties and data source options. + * @return The {@link DataFrameReader} to read from RDBMS + * @throws HoodieException + */ + private static DataFrameReader validatePropsAndGetDataFrameReader(final SparkSession session, + final TypedProperties properties) + throws HoodieException { + DataFrameReader dataFrameReader; + FSDataInputStream passwordFileStream = null; + try { + dataFrameReader = session.read().format("jdbc"); + dataFrameReader = dataFrameReader.option(Config.URL_PROP, properties.getString(Config.URL)); + dataFrameReader = dataFrameReader.option(Config.USER_PROP, properties.getString(Config.USER)); + dataFrameReader = dataFrameReader.option(Config.DRIVER_PROP, properties.getString(Config.DRIVER_CLASS)); + dataFrameReader = dataFrameReader + .option(Config.RDBMS_TABLE_PROP, properties.getString(Config.RDBMS_TABLE_NAME)); + + if (properties.containsKey(Config.PASSWORD)) { + LOG.info("Reading JDBC password from properties file...."); + dataFrameReader = dataFrameReader.option(Config.PASSWORD_PROP, properties.getString(Config.PASSWORD)); + } else if (properties.containsKey(Config.PASSWORD_FILE) + && !StringUtils.isNullOrEmpty(properties.getString(Config.PASSWORD_FILE))) { + LOG.info(String.format("Reading JDBC password from password file %s", properties.getString(Config.PASSWORD_FILE))); + FileSystem fileSystem = FileSystem.get(session.sparkContext().hadoopConfiguration()); + passwordFileStream = fileSystem.open(new Path(properties.getString(Config.PASSWORD_FILE))); + byte[] bytes = new byte[passwordFileStream.available()]; + passwordFileStream.read(bytes); + dataFrameReader = dataFrameReader.option(Config.PASSWORD_PROP, new String(bytes)); + } else { + throw new IllegalArgumentException(String.format("JDBCSource needs either a %s or %s to connect to RDBMS " + + "datasource", Config.PASSWORD_FILE, Config.PASSWORD)); + } + + addExtraJdbcOptions(properties, dataFrameReader); + + if (properties.getBoolean(Config.IS_INCREMENTAL)) { + DataSourceUtils.checkRequiredProperties(properties, Collections.singletonList(Config.INCREMENTAL_COLUMN)); Review comment: Would it be okay to do an additional query before fetching the dataset for this validation? ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/SqlQueryBuilder.java ########## @@ -0,0 +1,160 @@ +/* + * 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.hudi.utilities; + +import org.apache.hudi.common.util.StringUtils; + +/** + * Fluent SQL query builder. Review comment: By "fluent" I meant fluent builder pattern. This is not a library. However, I am reuing code from one of my projects where I used [http-rpc](https://github.com/HTTP-RPC/HTTP-RPC), specifically the [QueryBuilder](https://github.com/HTTP-RPC/HTTP-RPC/blob/master/httprpc-client/src/main/java/org/httprpc/sql/QueryBuilder.java) class. Please let me know how to attribute this, if needed. ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java ########## @@ -0,0 +1,326 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.SqlQueryBuilder; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.DataFrameReader; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.storage.StorageLevel; +import org.jetbrains.annotations.NotNull; Review comment: Not needed. I have removed it. Though, I found jetbrains annotations in [RequestHandler](https://github.com/apache/hudi/blob/master/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java#L48) in timeline service and [HoodieClusteringJob](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java#L38) in hudi-utilities. ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java ########## @@ -0,0 +1,326 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.SqlQueryBuilder; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.DataFrameReader; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.storage.StorageLevel; +import org.jetbrains.annotations.NotNull; + +import java.net.URI; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * Reads data from RDBMS data sources. + */ + +public class JdbcSource extends RowSource { + + private static final Logger LOG = LogManager.getLogger(JdbcSource.class); + private static final List<String> DB_LIMIT_CLAUSE = Arrays.asList("mysql", "postgresql", "h2"); + private static final String URI_JDBC_PREFIX = "jdbc:"; + + public JdbcSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + } + + /** + * Validates all user properties and prepares the {@link DataFrameReader} to read from RDBMS. + * + * @param session The {@link SparkSession}. + * @param properties The JDBC connection properties and data source options. + * @return The {@link DataFrameReader} to read from RDBMS + * @throws HoodieException + */ + private static DataFrameReader validatePropsAndGetDataFrameReader(final SparkSession session, + final TypedProperties properties) + throws HoodieException { + DataFrameReader dataFrameReader; + FSDataInputStream passwordFileStream = null; + try { + dataFrameReader = session.read().format("jdbc"); + dataFrameReader = dataFrameReader.option(Config.URL_PROP, properties.getString(Config.URL)); + dataFrameReader = dataFrameReader.option(Config.USER_PROP, properties.getString(Config.USER)); + dataFrameReader = dataFrameReader.option(Config.DRIVER_PROP, properties.getString(Config.DRIVER_CLASS)); + dataFrameReader = dataFrameReader + .option(Config.RDBMS_TABLE_PROP, properties.getString(Config.RDBMS_TABLE_NAME)); + + if (properties.containsKey(Config.PASSWORD)) { + LOG.info("Reading JDBC password from properties file...."); + dataFrameReader = dataFrameReader.option(Config.PASSWORD_PROP, properties.getString(Config.PASSWORD)); + } else if (properties.containsKey(Config.PASSWORD_FILE) + && !StringUtils.isNullOrEmpty(properties.getString(Config.PASSWORD_FILE))) { + LOG.info(String.format("Reading JDBC password from password file %s", properties.getString(Config.PASSWORD_FILE))); + FileSystem fileSystem = FileSystem.get(session.sparkContext().hadoopConfiguration()); + passwordFileStream = fileSystem.open(new Path(properties.getString(Config.PASSWORD_FILE))); + byte[] bytes = new byte[passwordFileStream.available()]; + passwordFileStream.read(bytes); + dataFrameReader = dataFrameReader.option(Config.PASSWORD_PROP, new String(bytes)); + } else { + throw new IllegalArgumentException(String.format("JDBCSource needs either a %s or %s to connect to RDBMS " + + "datasource", Config.PASSWORD_FILE, Config.PASSWORD)); + } + + addExtraJdbcOptions(properties, dataFrameReader); + + if (properties.getBoolean(Config.IS_INCREMENTAL)) { + DataSourceUtils.checkRequiredProperties(properties, Collections.singletonList(Config.INCREMENTAL_COLUMN)); + } + return dataFrameReader; + } catch (Exception e) { + throw new HoodieException(e); + } finally { + IOUtils.closeStream(passwordFileStream); + } + } + + /** + * Accepts spark JDBC options from the user in terms of EXTRA_OPTIONS adds them to {@link DataFrameReader} Example: In + * a normal spark code you would do something like: session.read.format('jdbc') .option(fetchSize,1000) + * .option(timestampFormat,"yyyy-mm-dd hh:mm:ss") + * <p> + * The way to pass these properties to HUDI is through the config file. Any property starting with + * hoodie.datasource.jdbc.extra.options. will be added. + * <p> + * Example: hoodie.datasource.jdbc.extra.options.fetchSize=100 + * hoodie.datasource.jdbc.extra.options.upperBound=1 + * hoodie.datasource.jdbc.extra.options.lowerBound=100 + * + * @param properties The JDBC connection properties and data source options. + * @param dataFrameReader The {@link DataFrameReader} to which data source options will be added. + */ + private static void addExtraJdbcOptions(TypedProperties properties, DataFrameReader dataFrameReader) { + Set<Object> objects = properties.keySet(); + for (Object property : objects) { + String prop = (String) property; + if (prop.startsWith(Config.EXTRA_OPTIONS)) { + String key = String.join("", prop.split(Config.EXTRA_OPTIONS)); + String value = properties.getString(prop); + if (!StringUtils.isNullOrEmpty(value)) { + LOG.info(String.format("Adding %s -> %s to jdbc options", key, value)); + dataFrameReader.option(key, value); + } + } + } + } + + @Override + protected Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkptStr, long sourceLimit) { + try { + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.URL, Config.DRIVER_CLASS, Config.USER, Config.RDBMS_TABLE_NAME, Config.IS_INCREMENTAL)); + return fetch(lastCkptStr, sourceLimit); + } catch (Exception e) { + LOG.error("Exception while running JDBCSource ", e); + return Pair.of(Option.empty(), null); + } + } + + /** + * Decide to do a full RDBMS table scan or an incremental scan based on the lastCkptStr. If previous checkpoint + * value exists then we do an incremental scan with a PPD query or else we do a full scan. In certain cases where the + * incremental query fails, we fallback to a full scan. + * + * @param lastCkptStr Last checkpoint. + * @return The pair of {@link Dataset} and current checkpoint. + */ + @NotNull + private Pair<Option<Dataset<Row>>, String> fetch(Option<String> lastCkptStr, long sourceLimit) { + Dataset<Row> dataset; + if (lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get())) { + dataset = incrementalFetch(lastCkptStr, sourceLimit); + } else { + LOG.info("No checkpoint references found. Doing a full rdbms table fetch"); + dataset = fullFetch(); + } + + if (props.containsKey(Config.STORAGE_LEVEL) && !StringUtils.isNullOrEmpty(props.getString(Config.STORAGE_LEVEL))) { Review comment: Simlpified in the latest update of the PR by using props.getString(key, defaultValue). ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java ########## @@ -0,0 +1,326 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.SqlQueryBuilder; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.DataFrameReader; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.storage.StorageLevel; +import org.jetbrains.annotations.NotNull; + +import java.net.URI; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * Reads data from RDBMS data sources. + */ + +public class JdbcSource extends RowSource { + + private static final Logger LOG = LogManager.getLogger(JdbcSource.class); + private static final List<String> DB_LIMIT_CLAUSE = Arrays.asList("mysql", "postgresql", "h2"); + private static final String URI_JDBC_PREFIX = "jdbc:"; + + public JdbcSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + } + + /** + * Validates all user properties and prepares the {@link DataFrameReader} to read from RDBMS. + * + * @param session The {@link SparkSession}. + * @param properties The JDBC connection properties and data source options. + * @return The {@link DataFrameReader} to read from RDBMS + * @throws HoodieException + */ + private static DataFrameReader validatePropsAndGetDataFrameReader(final SparkSession session, + final TypedProperties properties) + throws HoodieException { + DataFrameReader dataFrameReader; + FSDataInputStream passwordFileStream = null; + try { + dataFrameReader = session.read().format("jdbc"); + dataFrameReader = dataFrameReader.option(Config.URL_PROP, properties.getString(Config.URL)); + dataFrameReader = dataFrameReader.option(Config.USER_PROP, properties.getString(Config.USER)); + dataFrameReader = dataFrameReader.option(Config.DRIVER_PROP, properties.getString(Config.DRIVER_CLASS)); + dataFrameReader = dataFrameReader + .option(Config.RDBMS_TABLE_PROP, properties.getString(Config.RDBMS_TABLE_NAME)); + + if (properties.containsKey(Config.PASSWORD)) { + LOG.info("Reading JDBC password from properties file...."); + dataFrameReader = dataFrameReader.option(Config.PASSWORD_PROP, properties.getString(Config.PASSWORD)); + } else if (properties.containsKey(Config.PASSWORD_FILE) + && !StringUtils.isNullOrEmpty(properties.getString(Config.PASSWORD_FILE))) { + LOG.info(String.format("Reading JDBC password from password file %s", properties.getString(Config.PASSWORD_FILE))); + FileSystem fileSystem = FileSystem.get(session.sparkContext().hadoopConfiguration()); + passwordFileStream = fileSystem.open(new Path(properties.getString(Config.PASSWORD_FILE))); + byte[] bytes = new byte[passwordFileStream.available()]; + passwordFileStream.read(bytes); + dataFrameReader = dataFrameReader.option(Config.PASSWORD_PROP, new String(bytes)); + } else { + throw new IllegalArgumentException(String.format("JDBCSource needs either a %s or %s to connect to RDBMS " + + "datasource", Config.PASSWORD_FILE, Config.PASSWORD)); + } + + addExtraJdbcOptions(properties, dataFrameReader); + + if (properties.getBoolean(Config.IS_INCREMENTAL)) { + DataSourceUtils.checkRequiredProperties(properties, Collections.singletonList(Config.INCREMENTAL_COLUMN)); + } + return dataFrameReader; + } catch (Exception e) { + throw new HoodieException(e); + } finally { + IOUtils.closeStream(passwordFileStream); + } + } + + /** + * Accepts spark JDBC options from the user in terms of EXTRA_OPTIONS adds them to {@link DataFrameReader} Example: In + * a normal spark code you would do something like: session.read.format('jdbc') .option(fetchSize,1000) + * .option(timestampFormat,"yyyy-mm-dd hh:mm:ss") + * <p> + * The way to pass these properties to HUDI is through the config file. Any property starting with + * hoodie.datasource.jdbc.extra.options. will be added. + * <p> + * Example: hoodie.datasource.jdbc.extra.options.fetchSize=100 + * hoodie.datasource.jdbc.extra.options.upperBound=1 + * hoodie.datasource.jdbc.extra.options.lowerBound=100 + * + * @param properties The JDBC connection properties and data source options. + * @param dataFrameReader The {@link DataFrameReader} to which data source options will be added. + */ + private static void addExtraJdbcOptions(TypedProperties properties, DataFrameReader dataFrameReader) { + Set<Object> objects = properties.keySet(); + for (Object property : objects) { + String prop = (String) property; + if (prop.startsWith(Config.EXTRA_OPTIONS)) { + String key = String.join("", prop.split(Config.EXTRA_OPTIONS)); + String value = properties.getString(prop); + if (!StringUtils.isNullOrEmpty(value)) { + LOG.info(String.format("Adding %s -> %s to jdbc options", key, value)); + dataFrameReader.option(key, value); + } + } + } + } + + @Override + protected Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkptStr, long sourceLimit) { + try { + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.URL, Config.DRIVER_CLASS, Config.USER, Config.RDBMS_TABLE_NAME, Config.IS_INCREMENTAL)); + return fetch(lastCkptStr, sourceLimit); + } catch (Exception e) { + LOG.error("Exception while running JDBCSource ", e); + return Pair.of(Option.empty(), null); + } + } + + /** + * Decide to do a full RDBMS table scan or an incremental scan based on the lastCkptStr. If previous checkpoint + * value exists then we do an incremental scan with a PPD query or else we do a full scan. In certain cases where the + * incremental query fails, we fallback to a full scan. + * + * @param lastCkptStr Last checkpoint. + * @return The pair of {@link Dataset} and current checkpoint. + */ + @NotNull + private Pair<Option<Dataset<Row>>, String> fetch(Option<String> lastCkptStr, long sourceLimit) { + Dataset<Row> dataset; + if (lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get())) { + dataset = incrementalFetch(lastCkptStr, sourceLimit); + } else { + LOG.info("No checkpoint references found. Doing a full rdbms table fetch"); + dataset = fullFetch(); + } + + if (props.containsKey(Config.STORAGE_LEVEL) && !StringUtils.isNullOrEmpty(props.getString(Config.STORAGE_LEVEL))) { + dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL))); + } else { + dataset.persist(StorageLevel.MEMORY_AND_DISK_SER()); + } + + boolean isIncremental = props.getBoolean(Config.IS_INCREMENTAL); + Pair<Option<Dataset<Row>>, String> pair = Pair.of(Option.of(dataset), checkpoint(dataset, isIncremental)); + dataset.unpersist(); + + return pair; + } + + /** + * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint. + * + * @param lastCheckpoint Last checkpoint + * @return The {@link Dataset} after incremental fetch from RDBMS. + */ + @NotNull + private Dataset<Row> incrementalFetch(Option<String> lastCheckpoint, long sourceLimit) { + try { + final String ppdQuery = "(%s) rdbms_table"; + final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*") + .from(props.getString(Config.RDBMS_TABLE_NAME)) + .where(String.format(" %s > '%s'", props.getString(Config.INCREMENTAL_COLUMN), lastCheckpoint.get())); + + if (sourceLimit > 0) { + URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length())); + if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) { + queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit); + } + } + + String query = String.format(ppdQuery, queryBuilder.toString()); + LOG.error("PPD QUERY: " + query); + LOG.info(String.format("Referenced last checkpoint and prepared new predicate pushdown query for jdbc pull %s", query)); + + return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load(); + } catch (Exception e) { + LOG.error("Error while performing an incremental fetch. Not all database support the PPD query we generate to do an incremental scan", e); + LOG.warn("Falling back to full scan."); + + return fullFetch(); + } + } + + /** + * Does a full scan on the RDBMS data source. + * + * @return The {@link Dataset} after running full scan. + */ + private Dataset<Row> fullFetch() { + return validatePropsAndGetDataFrameReader(sparkSession, props).load(); + } + + private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental) { + try { + if (isIncremental) { + Column incrementalColumn = rowDataset.col(props.getString(Config.INCREMENTAL_COLUMN)); + final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first().getString(0); + LOG.info(String.format("Checkpointing column %s with value: %s ", incrementalColumn, max)); + return max; + } else { + return null; + } + } catch (Exception e) { + return null; + } + } + + /** + * Inner class with config keys. + */ + protected static class Config { + + /** + * {@value #URL} is the jdbc url for the Hoodie datasource. + */ + private static final String URL = "hoodie.datasource.jdbc.url"; Review comment: Done. -- 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]
