You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/05/05 12:32:06 UTC

[GitHub] [hudi] codope opened a new pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

codope opened a new pull request #2915:
URL: https://github.com/apache/hudi/pull/2915


   As discussed in [RFC-14](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+14+%3A+JDBC+incremental+puller), this change implements the first phase of JDBC incremental puller.
   It consists following changes:
   
   - JdbcSource: This class extends RowSource and implements
     fetchNextBatch(Option<String> lastCkptStr, long sourceLimit)
   
   - SqlQueryBuilder: A simple utility class to build sql queries fluently.
   
   - Implements two modes of fetching: full and incremental.
     Full is a complete scan of RDBMS table.
     Incremental is delta since last checkpoint.
     Incremental mode falls back to full fetch in case of any exception.
   
   - Caches the dataset after scan. Storage level is configurable.
   
   - Added unit tests to cover both incremental and full fetch.
   
   ## Committer checklist
   
    - [*] Has a corresponding JIRA in PR title & commit
    
    - [*] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [*] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-850826058


   > @codope : all feedback addressed? can we review. or do ping us here once its ready for review.
   
   @nsivabalan Almost all comments are addressed. One remaining is regarding the validations of incremental column. I had a question there https://github.com/apache/hudi/pull/2915#discussion_r633690635
   
   Could you please suggest how to proceed on that? As per current implementation, in case of non-existent column or mismatching column type, an exception will be thrown and puller will stop.


-- 
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:
users@infra.apache.org



[GitHub] [hudi] leesf commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627412205



##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    StringBuilder sqlBuilder = new StringBuilder();
+    sqlBuilder.append("select ");
+    sqlBuilder.append(String.join(", ", columns));
+
+    return new SqlQueryBuilder(sqlBuilder);
+  }
+
+  /**
+   * Appends a FROM clause to a query.
+   *
+   * @param tables The table names to select from.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder from(String... tables) {
+    if (tables == null || tables.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" from ");
+    sqlBuilder.append(String.join(", ", tables));

Review comment:
       would select from multi tables?

##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    StringBuilder sqlBuilder = new StringBuilder();
+    sqlBuilder.append("select ");
+    sqlBuilder.append(String.join(", ", columns));
+
+    return new SqlQueryBuilder(sqlBuilder);
+  }
+
+  /**
+   * Appends a FROM clause to a query.
+   *
+   * @param tables The table names to select from.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder from(String... tables) {
+    if (tables == null || tables.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" from ");
+    sqlBuilder.append(String.join(", ", tables));
+
+    return this;
+  }
+
+  /**
+   * Appends a JOIN clause to a query.
+   *
+   * @param table The table to join with.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder join(String table) {
+    if (StringUtils.isNullOrEmpty(table)) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" join ");
+    sqlBuilder.append(table);
+
+    return this;
+  }
+
+  /**
+   * Appends an ON clause to a query.
+   *
+   * @param predicate The predicate to join on.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder on(String predicate) {
+    if (StringUtils.isNullOrEmpty(predicate)) {
+      throw new IllegalArgumentException();

Review comment:
       ditto




-- 
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:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-861653740


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "877103f83dc9ea2ed3d8bffecd0d740c3dfc391a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "877103f83dc9ea2ed3d8bffecd0d740c3dfc391a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 877103f83dc9ea2ed3d8bffecd0d740c3dfc391a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r650811408



##########
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:
       @nsivabalan I have added more subtasks to this umbrella JIRA: https://issues.apache.org/jira/browse/HUDI-1859




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d78b858) into [master](https://codecov.io/gh/apache/hudi/commit/1db904a12ed295e08ded14bb5677dc02d76887a2?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1db904a) will **increase** coverage by `9.88%`.
   > The diff coverage is `92.74%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2915      +/-   ##
   ============================================
   + Coverage     54.22%   64.11%   +9.88%     
   + Complexity     3809      379    -3430     
   ============================================
     Files           488       56     -432     
     Lines         23574     2126   -21448     
     Branches       2510      247    -2263     
   ============================================
   - Hits          12783     1363   -11420     
   + Misses         9637      636    -9001     
   + Partials       1154      127    -1027     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `64.11% <92.74%> (-5.42%)` | `379.00 <34.00> (+5.00)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | `16.00 <16.00> (?)` | |
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `92.85% <92.85%> (ø)` | `18.00 <18.00> (?)` | |
   | [...s/deltastreamer/HoodieMultiTableDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllTXVsdGlUYWJsZURlbHRhU3RyZWFtZXIuamF2YQ==) | `0.00% <0.00%> (-76.20%)` | `0.00% <0.00%> (-19.00%)` | |
   | [...utilities/deltastreamer/TableExecutionContext.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvVGFibGVFeGVjdXRpb25Db250ZXh0LmphdmE=) | `0.00% <0.00%> (-65.00%)` | `0.00% <0.00%> (-9.00%)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `70.53% <0.00%> (-1.25%)` | `17.00% <0.00%> (-1.00%)` | |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `71.08% <0.00%> (ø)` | `55.00% <0.00%> (ø%)` | |
   | [...ava/org/apache/hudi/cli/commands/UtilsCommand.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL1V0aWxzQ29tbWFuZC5qYXZh) | | | |
   | [.../hadoop/realtime/RealtimeUnmergedRecordReader.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL3JlYWx0aW1lL1JlYWx0aW1lVW5tZXJnZWRSZWNvcmRSZWFkZXIuamF2YQ==) | | | |
   | [...org/apache/hudi/common/util/SpillableMapUtils.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvU3BpbGxhYmxlTWFwVXRpbHMuamF2YQ==) | | | |
   | [...rc/main/scala/org/apache/hudi/Spark3RowSerDe.scala](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3BhcmszL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvaHVkaS9TcGFyazNSb3dTZXJEZS5zY2FsYQ==) | | | |
   | ... and [432 more](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627927954



##########
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

Review comment:
       Document somewhere that during incremental fetch, this fetches rows with col val > checkpoint value (and not >=). Or in other words, exclusive. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] leesf commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627411500



##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();

Review comment:
       would add exception message

##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    StringBuilder sqlBuilder = new StringBuilder();
+    sqlBuilder.append("select ");
+    sqlBuilder.append(String.join(", ", columns));
+
+    return new SqlQueryBuilder(sqlBuilder);
+  }
+
+  /**
+   * Appends a FROM clause to a query.
+   *
+   * @param tables The table names to select from.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder from(String... tables) {
+    if (tables == null || tables.length == 0) {
+      throw new IllegalArgumentException();

Review comment:
       ditto

##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    StringBuilder sqlBuilder = new StringBuilder();
+    sqlBuilder.append("select ");
+    sqlBuilder.append(String.join(", ", columns));
+
+    return new SqlQueryBuilder(sqlBuilder);
+  }
+
+  /**
+   * Appends a FROM clause to a query.
+   *
+   * @param tables The table names to select from.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder from(String... tables) {
+    if (tables == null || tables.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" from ");
+    sqlBuilder.append(String.join(", ", tables));
+
+    return this;
+  }
+
+  /**
+   * Appends a JOIN clause to a query.
+   *
+   * @param table The table to join with.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder join(String table) {
+    if (StringUtils.isNullOrEmpty(table)) {
+      throw new IllegalArgumentException();

Review comment:
       ditto




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r650815907



##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    StringBuilder sqlBuilder = new StringBuilder();
+    sqlBuilder.append("select ");
+    sqlBuilder.append(String.join(", ", columns));
+
+    return new SqlQueryBuilder(sqlBuilder);
+  }
+
+  /**
+   * Appends a FROM clause to a query.
+   *
+   * @param tables The table names to select from.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder from(String... tables) {
+    if (tables == null || tables.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" from ");
+    sqlBuilder.append(String.join(", ", tables));

Review comment:
       Added a subtask to take it up after we land this PR.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce5c32) into [master](https://codecov.io/gh/apache/hudi/commit/8869b3b4184bbec4502e2e3f6fde0ea9260cf0b0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8869b3b) will **decrease** coverage by `46.11%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2915       +/-   ##
   ============================================
   - Coverage     54.79%   8.68%   -46.12%     
   + Complexity     3817      48     -3769     
   ============================================
     Files           483      56      -427     
     Lines         23347    2154    -21193     
     Branches       2491     257     -2234     
   ============================================
   - Hits          12794     187    -12607     
   + Misses         9399    1954     -7445     
   + Partials       1154      13     -1141     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `8.68% <0.00%> (-60.92%)` | `48.00 <0.00> (-330.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | ... and [464 more](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-864411567


   Thanks for your contribution. Merging in. 


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r650797408



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;

Review comment:
       We call checkpoint() only after fetching the dataset (#L177). If there was an exception during incrementalFetch() then it will fallback to fulFetch() if that's enabled (#L208). However, that check is before checkpoint() is called and if we throw an exception here then the sync will stop. I thought it's better to continue with an empty checkpoint in this case because we will hit this block only when incremental fetch is enabled and fetch completed, but something went wrong during checkpointing, so let the next round start over fresh. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce5c32) into [master](https://codecov.io/gh/apache/hudi/commit/8869b3b4184bbec4502e2e3f6fde0ea9260cf0b0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8869b3b) will **decrease** coverage by `1.13%`.
   > The diff coverage is `91.30%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2915      +/-   ##
   ============================================
   - Coverage     54.79%   53.66%   -1.14%     
   + Complexity     3817     3413     -404     
   ============================================
     Files           483      422      -61     
     Lines         23347    20007    -3340     
     Branches       2491     2086     -405     
   ============================================
   - Hits          12794    10737    -2057     
   + Misses         9399     8353    -1046     
   + Partials       1154      917     -237     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `39.53% <ø> (ø)` | `220.00 <ø> (ø)` | |
   | hudiclient | `∅ <ø> (∅)` | `0.00 <ø> (ø)` | |
   | hudicommon | `50.39% <ø> (ø)` | `1981.00 <ø> (ø)` | |
   | hudiflink | `63.11% <ø> (ø)` | `529.00 <ø> (ø)` | |
   | hudihadoopmr | `51.01% <ø> (ø)` | `266.00 <ø> (ø)` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `71.07% <91.30%> (+1.48%)` | `417.00 <38.00> (+39.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `90.81% <90.81%> (ø)` | `22.00 <22.00> (?)` | |
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | `16.00 <16.00> (?)` | |
   | [...in/java/org/apache/hudi/hive/SchemaDifference.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvU2NoZW1hRGlmZmVyZW5jZS5qYXZh) | | | |
   | [.../org/apache/hudi/hive/HoodieHiveSyncException.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSG9vZGllSGl2ZVN5bmNFeGNlcHRpb24uamF2YQ==) | | | |
   | [...3/internal/HoodieBulkInsertDataInternalWriter.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3BhcmszL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL3NwYXJrMy9pbnRlcm5hbC9Ib29kaWVCdWxrSW5zZXJ0RGF0YUludGVybmFsV3JpdGVyLmphdmE=) | | | |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | | | |
   | [...rc/main/scala/org/apache/hudi/Spark3RowSerDe.scala](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3BhcmszL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvaHVkaS9TcGFyazNSb3dTZXJEZS5zY2FsYQ==) | | | |
   | [...e/hudi/exception/HoodieDeltaStreamerException.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvZXhjZXB0aW9uL0hvb2RpZURlbHRhU3RyZWFtZXJFeGNlcHRpb24uamF2YQ==) | | | |
   | [...va/org/apache/hudi/hive/util/ColumnNameXLator.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db2x1bW5OYW1lWExhdG9yLmphdmE=) | | | |
   | [...main/java/org/apache/hudi/hive/HiveSyncConfig.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN5bmNDb25maWcuamF2YQ==) | | | |
   | ... and [59 more](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
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?




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r643894403



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;
+    }
+  }
+
+  /**
+   * 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.deltastreamer.jdbc.url";
+
+    private static final String URL_PROP = "url";
+
+    /**
+     * {@value #USER} is the username used for JDBC connection.
+     */
+    private static final String USER = "hoodie.deltastreamer.jdbc.user";
+
+    /**
+     * {@value #USER_PROP} used internally to build jdbc params.
+     */
+    private static final String USER_PROP = "user";
+
+    /**
+     * {@value #PASSWORD} is the password used for JDBC connection.
+     */
+    private static final String PASSWORD = "hoodie.deltastreamer.jdbc.password";
+
+    /**
+     * {@value #PASSWORD_FILE} is the base-path for the JDBC password file.
+     */
+    private static final String PASSWORD_FILE = "hoodie.deltastreamer.jdbc.password.file";
+
+    /**
+     * {@value #PASSWORD_PROP} used internally to build jdbc params.
+     */
+    private static final String PASSWORD_PROP = "password";
+
+    /**
+     * {@value #DRIVER_CLASS} used for JDBC connection.
+     */
+    private static final String DRIVER_CLASS = "hoodie.deltastreamer.jdbc.driver.class";
+
+    /**
+     * {@value #DRIVER_PROP} used internally to build jdbc params.
+     */
+    private static final String DRIVER_PROP = "driver";
+
+    /**
+     * {@value #RDBMS_TABLE_NAME} RDBMS table to pull.
+     */
+    private static final String RDBMS_TABLE_NAME = "hoodie.deltastreamer.jdbc.table.name";
+
+    /**
+     * {@value #RDBMS_TABLE_PROP} used internally for jdbc.
+     */
+    private static final String RDBMS_TABLE_PROP = "dbtable";
+
+    /**
+     * {@value #INCREMENTAL_COLUMN} if ran in incremental mode, this field will be used to pull new data incrementally.
+     */
+    private static final String INCREMENTAL_COLUMN = "hoodie.deltastreamer.jdbc.table.incremental.column.name";
+
+    /**
+     * {@value #IS_INCREMENTAL} will the JDBC source do an incremental pull?
+     */
+    private static final String IS_INCREMENTAL = "hoodie.deltastreamer.jdbc.incremental.pull";
+
+    /**
+     * {@value #EXTRA_OPTIONS} used to set any extra options the user specifies for jdbc.
+     */
+    private static final String EXTRA_OPTIONS = "hoodie.deltastreamer.jdbc.extra.options.";
+
+    /**
+     * {@value #STORAGE_LEVEL} is used to control the persistence level. Default value: MEMORY_AND_DISK_SER.
+     */
+    private static final String STORAGE_LEVEL = "hoodie.deltastreamer.jdbc.storage.level";
+
+    /**
+     * {@value #FALLBACK_TO_FULL_FETCH} is a boolean, which if set true, makes incremental fetch to fallback to full fetch in case of any error.
+     */
+    private static final String FALLBACK_TO_FULL_FETCH = "hoodie.deltastreamer.jdbc.incremental.fallback_to_full";

Review comment:
       if you consider my other comment "incr", then you could change this config to "hoodie.deltastreamer.jdbc.incr.fallback.to.full.fetch".
   1. incremental -> incr 
   2. replace "_" with "."
   3. suffix "fetch". 
   
   I mean, only (1) is debatable. but do fix other two. 

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;
+    }
+  }
+
+  /**
+   * 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.deltastreamer.jdbc.url";
+
+    private static final String URL_PROP = "url";
+
+    /**
+     * {@value #USER} is the username used for JDBC connection.
+     */
+    private static final String USER = "hoodie.deltastreamer.jdbc.user";
+
+    /**
+     * {@value #USER_PROP} used internally to build jdbc params.
+     */
+    private static final String USER_PROP = "user";
+
+    /**
+     * {@value #PASSWORD} is the password used for JDBC connection.
+     */
+    private static final String PASSWORD = "hoodie.deltastreamer.jdbc.password";
+
+    /**
+     * {@value #PASSWORD_FILE} is the base-path for the JDBC password file.
+     */
+    private static final String PASSWORD_FILE = "hoodie.deltastreamer.jdbc.password.file";
+
+    /**
+     * {@value #PASSWORD_PROP} used internally to build jdbc params.
+     */
+    private static final String PASSWORD_PROP = "password";
+
+    /**
+     * {@value #DRIVER_CLASS} used for JDBC connection.
+     */
+    private static final String DRIVER_CLASS = "hoodie.deltastreamer.jdbc.driver.class";
+
+    /**
+     * {@value #DRIVER_PROP} used internally to build jdbc params.
+     */
+    private static final String DRIVER_PROP = "driver";
+
+    /**
+     * {@value #RDBMS_TABLE_NAME} RDBMS table to pull.
+     */
+    private static final String RDBMS_TABLE_NAME = "hoodie.deltastreamer.jdbc.table.name";
+
+    /**
+     * {@value #RDBMS_TABLE_PROP} used internally for jdbc.
+     */
+    private static final String RDBMS_TABLE_PROP = "dbtable";
+
+    /**
+     * {@value #INCREMENTAL_COLUMN} if ran in incremental mode, this field will be used to pull new data incrementally.
+     */
+    private static final String INCREMENTAL_COLUMN = "hoodie.deltastreamer.jdbc.table.incremental.column.name";

Review comment:
       for brevity purposes, can we do "incr" instead of "incremental". 

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;

Review comment:
       I thought we might throw an exception here? fallback to full fetch is controlled elsewhere right. can you please clarify.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r647525435



##########
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:
       sure. I am okay doing a followup in the interest of landing this PR. @codope : guess you have quite a few followups. Can you link the follow up jira here. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r643976142



##########
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:
       yes. may be we can try an optimization. if hudi table does not exist, we can do the validation. i.e. only for first time. but for subsequent times, we can avoid this validation. @vinothchandar : is there any guidelines in general we follow for other sources in general for similar type of validations. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r643892144



##########
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

Review comment:
       can you please clarify this.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r633696647



##########
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).




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r651418719



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;

Review comment:
       @n3nash : any thoughts here. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r651418549



##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJdbcSource.java
##########
@@ -0,0 +1,442 @@
+/*
+ * 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.common.config.TypedProperties;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.storage.StorageLevel;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.clearAndInsert;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.close;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.count;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.insert;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.update;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests {@link JdbcSource}.
+ */
+public class TestJdbcSource extends UtilitiesTestBase {
+
+  private static final TypedProperties PROPS = new TypedProperties();
+  private static final HoodieTestDataGenerator DATA_GENERATOR = new HoodieTestDataGenerator();
+  private static Connection connection;
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.url", "jdbc:h2:mem:test_mem");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.driver.class", "org.h2.Driver");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.user", "test");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.password", "jdbc");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.name", "triprec");
+    connection = DriverManager.getConnection("jdbc:h2:mem:test_mem", "test", "jdbc");
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+    close(connection);
+  }
+
+  @Test
+  public void testSingleCommit() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      int numRecords = 100;
+      String commitTime = "000";
+
+      // Insert 100 records with commit time
+      clearAndInsert(commitTime, numRecords, connection, DATA_GENERATOR, PROPS);
+
+      // Validate if we have specified records in db
+      assertEquals(numRecords, count(connection, "triprec"));
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), numRecords).getBatch().get();
+      assertEquals(numRecords, rowDataset.count());
+    } catch (SQLException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testInsertAndUpdate() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      final String commitTime = "000";
+      final int numRecords = 100;
+
+      // Add 100 records. Update half of them with commit time "007".
+      update("007",
+          clearAndInsert(commitTime, numRecords, connection, DATA_GENERATOR, PROPS)
+              .stream()
+              .limit(50)
+              .collect(Collectors.toList()),
+          connection, DATA_GENERATOR, PROPS
+      );
+      // Check if database has 100 records
+      assertEquals(numRecords, count(connection, "triprec"));
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 100).getBatch().get();
+      assertEquals(100, rowDataset.count());
+
+      Dataset<Row> firstCommit = rowDataset.where("commit_time=000");
+      assertEquals(50, firstCommit.count());
+
+      Dataset<Row> secondCommit = rowDataset.where("commit_time=007");
+      assertEquals(50, secondCommit.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testTwoCommits() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10).getBatch().get();
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Add 10 records with commit time 001
+      insert("001", 5, connection, DATA_GENERATOR, PROPS);
+      rowDataset = runSource(Option.empty(), 15).getBatch().get();
+      assertEquals(15, rowDataset.count());
+      assertEquals(5, rowDataset.where("commit_time=001").count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Start second commit and check if all records are pulled
+      rowDataset = runSource(Option.empty(), 15).getBatch().get();
+      assertEquals(15, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 10);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(10, rowDataset.count());
+
+      // Add 10 records with commit time "001"
+      insert("001", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 10).getBatch().get();
+      assertEquals(10, rowDataset.count());
+      assertEquals(10, rowDataset.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithNoMatchingRows() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 10);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(10, rowDataset.count());
+
+      // Start incremental scan
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 10).getBatch().get();
+      assertEquals(0, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWhenTableRecordsMoreThanSourceLimit() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "id");
+
+    try {
+      // Add 100 records with commit time "000"
+      clearAndInsert("000", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 100);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(100, rowDataset.count());
+
+      // Add 100 records with commit time "001"
+      insert("001", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan. Now there are 100 more records but with sourceLimit set to 60, only fetch 60 records should be fetched.
+      // Those 50 records should be of the commit_time=001 because records with commit_time=000 have already been processed.
+      batch = runSource(Option.of(batch.getCheckpointForNextBatch()), 60);
+      rowDataset = batch.getBatch().get();
+      assertEquals(60, rowDataset.count());
+      assertEquals(60, rowDataset.where("commit_time=001").count());
+      // No more records added, but sourceLimit is now set to 75. Still, only the remaining 40 records should be fetched.
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 75).getBatch().get();
+      assertEquals(40, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWhenLastCheckpointMoreThanTableRecords() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "id");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 100);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(100, rowDataset.count());
+      assertEquals("100", batch.getCheckpointForNextBatch());
+
+      // Add 10 records with commit time "001"
+      insert("001", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan. With checkpoint greater than the number of records, there should not be any dataset to fetch.
+      batch = runSource(Option.of("200"), 50);
+      rowDataset = batch.getBatch().get();
+      assertEquals(0, rowDataset.count());

Review comment:
       got it.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d78b858) into [master](https://codecov.io/gh/apache/hudi/commit/1db904a12ed295e08ded14bb5677dc02d76887a2?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1db904a) will **decrease** coverage by `0.38%`.
   > The diff coverage is `92.74%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2915      +/-   ##
   ============================================
   - Coverage     54.22%   53.83%   -0.39%     
   - Complexity     3809     3815       +6     
   ============================================
     Files           488      490       +2     
     Lines         23574    23698     +124     
     Branches       2510     2520      +10     
   ============================================
   - Hits          12783    12759      -24     
   - Misses         9637     9791     +154     
   + Partials       1154     1148       -6     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `39.53% <ø> (ø)` | `220.00 <ø> (ø)` | |
   | hudiclient | `∅ <ø> (∅)` | `0.00 <ø> (ø)` | |
   | hudicommon | `50.41% <ø> (+0.04%)` | `1976.00 <ø> (+1.00)` | |
   | hudiflink | `58.98% <ø> (ø)` | `538.00 <ø> (ø)` | |
   | hudihadoopmr | `50.93% <ø> (ø)` | `259.00 <ø> (ø)` | |
   | hudisparkdatasource | `73.33% <ø> (ø)` | `237.00 <ø> (ø)` | |
   | hudisync | `46.73% <ø> (ø)` | `144.00 <ø> (ø)` | |
   | huditimelineservice | `64.36% <ø> (ø)` | `62.00 <ø> (ø)` | |
   | hudiutilities | `64.11% <92.74%> (-5.42%)` | `379.00 <34.00> (+5.00)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | `16.00 <16.00> (?)` | |
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `92.85% <92.85%> (ø)` | `18.00 <18.00> (?)` | |
   | [...s/deltastreamer/HoodieMultiTableDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllTXVsdGlUYWJsZURlbHRhU3RyZWFtZXIuamF2YQ==) | `0.00% <0.00%> (-76.20%)` | `0.00% <0.00%> (-19.00%)` | |
   | [...utilities/deltastreamer/TableExecutionContext.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvVGFibGVFeGVjdXRpb25Db250ZXh0LmphdmE=) | `0.00% <0.00%> (-65.00%)` | `0.00% <0.00%> (-9.00%)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `70.53% <0.00%> (-1.25%)` | `17.00% <0.00%> (-1.00%)` | |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `71.08% <0.00%> (ø)` | `55.00% <0.00%> (ø%)` | |
   | [...e/hudi/common/table/log/HoodieLogFormatWriter.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGb3JtYXRXcml0ZXIuamF2YQ==) | `79.68% <0.00%> (+1.56%)` | `26.00% <0.00%> (ø%)` | |
   | [...ache/hudi/common/fs/inline/InMemoryFileSystem.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `89.65% <0.00%> (+10.34%)` | `16.00% <0.00%> (+1.00%)` | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] hudi-bot edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-861653740


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "877103f83dc9ea2ed3d8bffecd0d740c3dfc391a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=186",
       "triggerID" : "877103f83dc9ea2ed3d8bffecd0d740c3dfc391a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 877103f83dc9ea2ed3d8bffecd0d740c3dfc391a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=186) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r650804454



##########
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

Review comment:
       The idea is to support extra options that are available for jdbc source in spark. Specifically, upperBound/lowerBound could be helpful in future when reading in parallel. We can use the incremental column as the partition and set numPartitions, lowerBound, upperBound for data loading parallelly.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r650783802



##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJdbcSource.java
##########
@@ -0,0 +1,442 @@
+/*
+ * 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.common.config.TypedProperties;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.storage.StorageLevel;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.clearAndInsert;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.close;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.count;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.insert;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.update;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests {@link JdbcSource}.
+ */
+public class TestJdbcSource extends UtilitiesTestBase {
+
+  private static final TypedProperties PROPS = new TypedProperties();
+  private static final HoodieTestDataGenerator DATA_GENERATOR = new HoodieTestDataGenerator();
+  private static Connection connection;
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.url", "jdbc:h2:mem:test_mem");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.driver.class", "org.h2.Driver");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.user", "test");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.password", "jdbc");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.name", "triprec");
+    connection = DriverManager.getConnection("jdbc:h2:mem:test_mem", "test", "jdbc");
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+    close(connection);
+  }
+
+  @Test
+  public void testSingleCommit() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      int numRecords = 100;
+      String commitTime = "000";
+
+      // Insert 100 records with commit time
+      clearAndInsert(commitTime, numRecords, connection, DATA_GENERATOR, PROPS);
+
+      // Validate if we have specified records in db
+      assertEquals(numRecords, count(connection, "triprec"));
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), numRecords).getBatch().get();
+      assertEquals(numRecords, rowDataset.count());
+    } catch (SQLException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testInsertAndUpdate() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      final String commitTime = "000";
+      final int numRecords = 100;
+
+      // Add 100 records. Update half of them with commit time "007".
+      update("007",
+          clearAndInsert(commitTime, numRecords, connection, DATA_GENERATOR, PROPS)
+              .stream()
+              .limit(50)
+              .collect(Collectors.toList()),
+          connection, DATA_GENERATOR, PROPS
+      );
+      // Check if database has 100 records
+      assertEquals(numRecords, count(connection, "triprec"));
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 100).getBatch().get();
+      assertEquals(100, rowDataset.count());
+
+      Dataset<Row> firstCommit = rowDataset.where("commit_time=000");
+      assertEquals(50, firstCommit.count());
+
+      Dataset<Row> secondCommit = rowDataset.where("commit_time=007");
+      assertEquals(50, secondCommit.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testTwoCommits() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10).getBatch().get();
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Add 10 records with commit time 001
+      insert("001", 5, connection, DATA_GENERATOR, PROPS);
+      rowDataset = runSource(Option.empty(), 15).getBatch().get();
+      assertEquals(15, rowDataset.count());
+      assertEquals(5, rowDataset.where("commit_time=001").count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Start second commit and check if all records are pulled
+      rowDataset = runSource(Option.empty(), 15).getBatch().get();
+      assertEquals(15, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 10);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(10, rowDataset.count());
+
+      // Add 10 records with commit time "001"
+      insert("001", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 10).getBatch().get();
+      assertEquals(10, rowDataset.count());
+      assertEquals(10, rowDataset.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithNoMatchingRows() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 10);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(10, rowDataset.count());
+
+      // Start incremental scan
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 10).getBatch().get();
+      assertEquals(0, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWhenTableRecordsMoreThanSourceLimit() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "id");
+
+    try {
+      // Add 100 records with commit time "000"
+      clearAndInsert("000", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 100);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(100, rowDataset.count());
+
+      // Add 100 records with commit time "001"
+      insert("001", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan. Now there are 100 more records but with sourceLimit set to 60, only fetch 60 records should be fetched.
+      // Those 50 records should be of the commit_time=001 because records with commit_time=000 have already been processed.
+      batch = runSource(Option.of(batch.getCheckpointForNextBatch()), 60);
+      rowDataset = batch.getBatch().get();
+      assertEquals(60, rowDataset.count());
+      assertEquals(60, rowDataset.where("commit_time=001").count());
+      // No more records added, but sourceLimit is now set to 75. Still, only the remaining 40 records should be fetched.
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 75).getBatch().get();
+      assertEquals(40, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWhenLastCheckpointMoreThanTableRecords() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "id");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 100);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(100, rowDataset.count());
+      assertEquals("100", batch.getCheckpointForNextBatch());
+
+      // Add 10 records with commit time "001"
+      insert("001", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan. With checkpoint greater than the number of records, there should not be any dataset to fetch.
+      batch = runSource(Option.of("200"), 50);
+      rowDataset = batch.getBatch().get();
+      assertEquals(0, rowDataset.count());

Review comment:
       It will be 200. The last known checkpoint as it is being passed by DeltaSync. JdbcSource assumes that DeltaSync is sending the right checkpoint based on commit timeline and keep the last checkpoint in case dataset count is 0. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-833988217


   Can we add tests for these scenarios
   1. no matching rows with incremental fetch. lets say everything is caught up. 
   2. records in source table range from 0 to 100. you set checkpoint as 200. again, this should return empty set. 
   3. records in source table range from 100 to 200. checkpoint is set to 50 and source limit is 50. 
   4. Do we have tests when source table has more records than source limit. something like (3) above. 
   5. Write one long running test (10 iterations of incremental fetch). 
   
   Orthogonal point: 
   Did we consider option to renaming columns when inserting to hudi. not required in this diff. but in general. 


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076






-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076






-- 
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:
users@infra.apache.org



[GitHub] [hudi] leesf commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627418429



##########
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);

Review comment:
       info




-- 
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:
users@infra.apache.org



[GitHub] [hudi] vinothchandar commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-833746933


   Great suggestions @leesf ! We can file followups for those. 


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce5c32) into [master](https://codecov.io/gh/apache/hudi/commit/8869b3b4184bbec4502e2e3f6fde0ea9260cf0b0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8869b3b) will **increase** coverage by `0.22%`.
   > The diff coverage is `91.30%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2915      +/-   ##
   ============================================
   + Coverage     54.79%   55.02%   +0.22%     
   - Complexity     3817     3856      +39     
   ============================================
     Files           483      485       +2     
     Lines         23347    23485     +138     
     Branches       2491     2507      +16     
   ============================================
   + Hits          12794    12922     +128     
   - Misses         9399     9400       +1     
   - Partials       1154     1163       +9     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `39.53% <ø> (ø)` | `220.00 <ø> (ø)` | |
   | hudiclient | `∅ <ø> (∅)` | `0.00 <ø> (ø)` | |
   | hudicommon | `50.39% <ø> (ø)` | `1981.00 <ø> (ø)` | |
   | hudiflink | `63.11% <ø> (ø)` | `529.00 <ø> (ø)` | |
   | hudihadoopmr | `51.01% <ø> (ø)` | `266.00 <ø> (ø)` | |
   | hudisparkdatasource | `73.33% <ø> (ø)` | `237.00 <ø> (ø)` | |
   | hudisync | `46.44% <ø> (ø)` | `144.00 <ø> (ø)` | |
   | huditimelineservice | `64.36% <ø> (ø)` | `62.00 <ø> (ø)` | |
   | hudiutilities | `71.07% <91.30%> (+1.48%)` | `417.00 <38.00> (+39.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `90.81% <90.81%> (ø)` | `22.00 <22.00> (?)` | |
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | `16.00 <16.00> (?)` | |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `71.18% <0.00%> (+0.33%)` | `56.00% <0.00%> (+1.00%)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `72.83% <0.00%> (+0.39%)` | `22.00% <0.00%> (ø%)` | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] hudi-bot edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-861653740






-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (877103f) into [master](https://codecov.io/gh/apache/hudi/commit/769dd2d7c98558146eb4accb75b6d8e339ae6e0f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (769dd2d) will **increase** coverage by `4.36%`.
   > The diff coverage is `91.17%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2915      +/-   ##
   ============================================
   + Coverage     50.04%   54.40%   +4.36%     
   + Complexity     3685      444    -3241     
   ============================================
     Files           526       72     -454     
     Lines         25466     3016   -22450     
     Branches       2886      375    -2511     
   ============================================
   - Hits          12744     1641   -11103     
   + Misses        11454     1221   -10233     
   + Partials       1268      154    -1114     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `∅ <ø> (∅)` | |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `6.79% <ø> (-44.66%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `72.30% <91.17%> (+63.21%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `90.62% <90.62%> (ø)` | |
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/hive/HiveSyncConfig.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN5bmNDb25maWcuamF2YQ==) | `0.00% <0.00%> (-97.83%)` | :arrow_down: |
   | [.../apache/hudi/hive/MultiPartKeysValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTXVsdGlQYXJ0S2V5c1ZhbHVlRXh0cmFjdG9yLmphdmE=) | `0.00% <0.00%> (-90.91%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/SchemaDifference.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvU2NoZW1hRGlmZmVyZW5jZS5qYXZh) | `0.00% <0.00%> (-84.85%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/util/ConfigUtils.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db25maWdVdGlscy5qYXZh) | `0.00% <0.00%> (-73.92%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/HoodieHiveClient.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSG9vZGllSGl2ZUNsaWVudC5qYXZh) | `0.00% <0.00%> (-71.61%)` | :arrow_down: |
   | [...va/org/apache/hudi/hive/util/ColumnNameXLator.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db2x1bW5OYW1lWExhdG9yLmphdmE=) | `0.00% <0.00%> (-70.00%)` | :arrow_down: |
   | ... and [506 more](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r633697075



##########
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r643884770



##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java
##########
@@ -1591,6 +1596,45 @@ public void testCsvDFSSourceNoHeaderWithSchemaProviderAndTransformer() throws Ex
     testCsvDFSSource(false, '\t', true, Collections.singletonList(TripsWithDistanceTransformer.class.getName()));
   }
 
+  @Test
+  public void testIncrementalFetchInContinuousMode() {

Review comment:
       minor. rename to "testJDBCSourceIncremental....." 

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);

Review comment:
       minor. Can we do new HoodieException(msg, e). would be good to always give some context about where exception is thrown. we can infer from stacktrace, but msg would be nice too. 

##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJdbcSource.java
##########
@@ -0,0 +1,442 @@
+/*
+ * 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.common.config.TypedProperties;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.storage.StorageLevel;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.clearAndInsert;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.close;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.count;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.insert;
+import static org.apache.hudi.utilities.testutils.JdbcTestUtils.update;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests {@link JdbcSource}.
+ */
+public class TestJdbcSource extends UtilitiesTestBase {
+
+  private static final TypedProperties PROPS = new TypedProperties();
+  private static final HoodieTestDataGenerator DATA_GENERATOR = new HoodieTestDataGenerator();
+  private static Connection connection;
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.url", "jdbc:h2:mem:test_mem");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.driver.class", "org.h2.Driver");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.user", "test");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.password", "jdbc");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.name", "triprec");
+    connection = DriverManager.getConnection("jdbc:h2:mem:test_mem", "test", "jdbc");
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+    close(connection);
+  }
+
+  @Test
+  public void testSingleCommit() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      int numRecords = 100;
+      String commitTime = "000";
+
+      // Insert 100 records with commit time
+      clearAndInsert(commitTime, numRecords, connection, DATA_GENERATOR, PROPS);
+
+      // Validate if we have specified records in db
+      assertEquals(numRecords, count(connection, "triprec"));
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), numRecords).getBatch().get();
+      assertEquals(numRecords, rowDataset.count());
+    } catch (SQLException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testInsertAndUpdate() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      final String commitTime = "000";
+      final int numRecords = 100;
+
+      // Add 100 records. Update half of them with commit time "007".
+      update("007",
+          clearAndInsert(commitTime, numRecords, connection, DATA_GENERATOR, PROPS)
+              .stream()
+              .limit(50)
+              .collect(Collectors.toList()),
+          connection, DATA_GENERATOR, PROPS
+      );
+      // Check if database has 100 records
+      assertEquals(numRecords, count(connection, "triprec"));
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 100).getBatch().get();
+      assertEquals(100, rowDataset.count());
+
+      Dataset<Row> firstCommit = rowDataset.where("commit_time=000");
+      assertEquals(50, firstCommit.count());
+
+      Dataset<Row> secondCommit = rowDataset.where("commit_time=007");
+      assertEquals(50, secondCommit.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testTwoCommits() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10).getBatch().get();
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Add 10 records with commit time 001
+      insert("001", 5, connection, DATA_GENERATOR, PROPS);
+      rowDataset = runSource(Option.empty(), 15).getBatch().get();
+      assertEquals(15, rowDataset.count());
+      assertEquals(5, rowDataset.where("commit_time=001").count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Start second commit and check if all records are pulled
+      rowDataset = runSource(Option.empty(), 15).getBatch().get();
+      assertEquals(15, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 10);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(10, rowDataset.count());
+
+      // Add 10 records with commit time "001"
+      insert("001", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 10).getBatch().get();
+      assertEquals(10, rowDataset.count());
+      assertEquals(10, rowDataset.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithNoMatchingRows() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 10);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(10, rowDataset.count());
+
+      // Start incremental scan
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 10).getBatch().get();
+      assertEquals(0, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWhenTableRecordsMoreThanSourceLimit() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "id");
+
+    try {
+      // Add 100 records with commit time "000"
+      clearAndInsert("000", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 100);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(100, rowDataset.count());
+
+      // Add 100 records with commit time "001"
+      insert("001", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan. Now there are 100 more records but with sourceLimit set to 60, only fetch 60 records should be fetched.
+      // Those 50 records should be of the commit_time=001 because records with commit_time=000 have already been processed.
+      batch = runSource(Option.of(batch.getCheckpointForNextBatch()), 60);
+      rowDataset = batch.getBatch().get();
+      assertEquals(60, rowDataset.count());
+      assertEquals(60, rowDataset.where("commit_time=001").count());
+      // No more records added, but sourceLimit is now set to 75. Still, only the remaining 40 records should be fetched.
+      rowDataset = runSource(Option.of(batch.getCheckpointForNextBatch()), 75).getBatch().get();
+      assertEquals(40, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWhenLastCheckpointMoreThanTableRecords() {
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.deltastreamer.jdbc.table.incremental.column.name", "id");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start JdbcSource
+      InputBatch<Dataset<Row>> batch = runSource(Option.empty(), 100);
+      Dataset<Row> rowDataset = batch.getBatch().get();
+      assertEquals(100, rowDataset.count());
+      assertEquals("100", batch.getCheckpointForNextBatch());
+
+      // Add 10 records with commit time "001"
+      insert("001", 100, connection, DATA_GENERATOR, PROPS);
+
+      // Start incremental scan. With checkpoint greater than the number of records, there should not be any dataset to fetch.
+      batch = runSource(Option.of("200"), 50);
+      rowDataset = batch.getBatch().get();
+      assertEquals(0, rowDataset.count());

Review comment:
       just for my understanding. what will be the checkpoint(batch.getCheckpointForNextBatch()) value after this? will it stay w/ 200 or reset to last known checkpoint for the source. 

##########
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())) {

Review comment:
       got it.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce5c32) into [master](https://codecov.io/gh/apache/hudi/commit/8869b3b4184bbec4502e2e3f6fde0ea9260cf0b0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8869b3b) will **increase** coverage by `0.22%`.
   > The diff coverage is `91.30%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2915      +/-   ##
   ============================================
   + Coverage     54.79%   55.02%   +0.22%     
   - Complexity     3817     3856      +39     
   ============================================
     Files           483      485       +2     
     Lines         23347    23485     +138     
     Branches       2491     2507      +16     
   ============================================
   + Hits          12794    12922     +128     
   - Misses         9399     9400       +1     
   - Partials       1154     1163       +9     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `39.53% <ø> (ø)` | `220.00 <ø> (ø)` | |
   | hudiclient | `∅ <ø> (∅)` | `0.00 <ø> (ø)` | |
   | hudicommon | `50.39% <ø> (ø)` | `1981.00 <ø> (ø)` | |
   | hudiflink | `63.11% <ø> (ø)` | `529.00 <ø> (ø)` | |
   | hudihadoopmr | `51.01% <ø> (ø)` | `266.00 <ø> (ø)` | |
   | hudisparkdatasource | `73.33% <ø> (ø)` | `237.00 <ø> (ø)` | |
   | hudisync | `46.44% <ø> (ø)` | `144.00 <ø> (ø)` | |
   | huditimelineservice | `64.36% <ø> (ø)` | `62.00 <ø> (ø)` | |
   | hudiutilities | `71.07% <91.30%> (+1.48%)` | `417.00 <38.00> (+39.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `90.81% <90.81%> (ø)` | `22.00 <22.00> (?)` | |
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | `16.00 <16.00> (?)` | |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `71.18% <0.00%> (+0.33%)` | `56.00% <0.00%> (+1.00%)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `72.83% <0.00%> (+0.39%)` | `22.00% <0.00%> (ø%)` | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r633693469



##########
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.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] hudi-bot edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-861653740


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "877103f83dc9ea2ed3d8bffecd0d740c3dfc391a",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=186",
       "triggerID" : "877103f83dc9ea2ed3d8bffecd0d740c3dfc391a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 877103f83dc9ea2ed3d8bffecd0d740c3dfc391a Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=186) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ce5c32) into [master](https://codecov.io/gh/apache/hudi/commit/8869b3b4184bbec4502e2e3f6fde0ea9260cf0b0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8869b3b) will **increase** coverage by `16.27%`.
   > The diff coverage is `91.30%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2915       +/-   ##
   =============================================
   + Coverage     54.79%   71.07%   +16.27%     
   + Complexity     3817      417     -3400     
   =============================================
     Files           483       56      -427     
     Lines         23347     2154    -21193     
     Branches       2491      257     -2234     
   =============================================
   - Hits          12794     1531    -11263     
   + Misses         9399      480     -8919     
   + Partials       1154      143     -1011     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `71.07% <91.30%> (+1.48%)` | `417.00 <38.00> (+39.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `90.81% <90.81%> (ø)` | `22.00 <22.00> (?)` | |
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | `16.00 <16.00> (?)` | |
   | [...che/hudi/common/table/timeline/TimelineLayout.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL1RpbWVsaW5lTGF5b3V0LmphdmE=) | | | |
   | [...apache/hudi/common/util/collection/RocksDBDAO.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvY29sbGVjdGlvbi9Sb2Nrc0RCREFPLmphdmE=) | | | |
   | [...he/hudi/table/format/cow/ParquetDecimalVector.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS90YWJsZS9mb3JtYXQvY293L1BhcnF1ZXREZWNpbWFsVmVjdG9yLmphdmE=) | | | |
   | [...rg/apache/hudi/common/fs/inline/InLineFSUtils.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9JbkxpbmVGU1V0aWxzLmphdmE=) | | | |
   | [...di-cli/src/main/java/org/apache/hudi/cli/Main.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL01haW4uamF2YQ==) | | | |
   | [...mmon/table/log/AbstractHoodieLogRecordScanner.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9BYnN0cmFjdEhvb2RpZUxvZ1JlY29yZFNjYW5uZXIuamF2YQ==) | | | |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | | | |
   | [...rg/apache/hudi/common/bloom/SimpleBloomFilter.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2Jsb29tL1NpbXBsZUJsb29tRmlsdGVyLmphdmE=) | | | |
   | ... and [425 more](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] vinothchandar commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r647135255



##########
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:
       I think we can make this a follow on sub-task. or just introduce a flag for this validation in this PR. But my suggestion would be just keep it enabled/disabled, without introducing complexity of detecting the first time. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (877103f) into [master](https://codecov.io/gh/apache/hudi/commit/769dd2d7c98558146eb4accb75b6d8e339ae6e0f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (769dd2d) will **increase** coverage by `4.36%`.
   > The diff coverage is `91.17%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2915      +/-   ##
   ============================================
   + Coverage     50.04%   54.40%   +4.36%     
   + Complexity     3685      444    -3241     
   ============================================
     Files           526       72     -454     
     Lines         25466     3016   -22450     
     Branches       2886      375    -2511     
   ============================================
   - Hits          12744     1641   -11103     
   + Misses        11454     1221   -10233     
   + Partials       1268      154    -1114     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `?` | |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `6.79% <ø> (-44.66%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `72.30% <91.17%> (+63.21%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `90.62% <90.62%> (ø)` | |
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `92.50% <92.50%> (ø)` | |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/hive/HiveSyncConfig.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN5bmNDb25maWcuamF2YQ==) | `0.00% <0.00%> (-97.83%)` | :arrow_down: |
   | [.../apache/hudi/hive/MultiPartKeysValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTXVsdGlQYXJ0S2V5c1ZhbHVlRXh0cmFjdG9yLmphdmE=) | `0.00% <0.00%> (-90.91%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/SchemaDifference.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvU2NoZW1hRGlmZmVyZW5jZS5qYXZh) | `0.00% <0.00%> (-84.85%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/util/ConfigUtils.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db25maWdVdGlscy5qYXZh) | `0.00% <0.00%> (-73.92%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/HoodieHiveClient.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSG9vZGllSGl2ZUNsaWVudC5qYXZh) | `0.00% <0.00%> (-71.61%)` | :arrow_down: |
   | [...va/org/apache/hudi/hive/util/ColumnNameXLator.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db2x1bW5OYW1lWExhdG9yLmphdmE=) | `0.00% <0.00%> (-70.00%)` | :arrow_down: |
   | ... and [506 more](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r654115003



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;

Review comment:
       Got a chance to sync up w/ Nishith. We feel that we should just throw here and let user take a look at what's happening. Bcoz, if checkpoint cannot be parsed, it's an unexpected scenario.  How do we know checkpoint parsing will succeed in next round w/o taking any explicit action. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] vinothchandar commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627987063



##########
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:
       is Fluent an external library? are you reusing code from somewhere? if so, we need to attribute this. 

##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    StringBuilder sqlBuilder = new StringBuilder();
+    sqlBuilder.append("select ");
+    sqlBuilder.append(String.join(", ", columns));
+
+    return new SqlQueryBuilder(sqlBuilder);
+  }
+
+  /**
+   * Appends a FROM clause to a query.
+   *
+   * @param tables The table names to select from.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder from(String... tables) {
+    if (tables == null || tables.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" from ");
+    sqlBuilder.append(String.join(", ", tables));
+
+    return this;
+  }
+
+  /**
+   * Appends a JOIN clause to a query.
+   *
+   * @param table The table to join with.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder join(String table) {
+    if (StringUtils.isNullOrEmpty(table)) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" join ");
+    sqlBuilder.append(table);
+

Review comment:
       nit: remove redundant newline? (here and everywhere else?)

##########
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.
+ * Current support for: SELECT, FROM, JOIN, ON, WHERE, ORDER BY, LIMIT clauses.
+ */
+public class SqlQueryBuilder {
+
+  private StringBuilder sqlBuilder;
+
+  private SqlQueryBuilder(StringBuilder sqlBuilder) {
+    this.sqlBuilder = sqlBuilder;
+  }
+
+  /**
+   * Creates a SELECT query.
+   *
+   * @param columns The column names to select.
+   * @return The new {@link SqlQueryBuilder} instance.
+   */
+  public static SqlQueryBuilder select(String... columns) {
+    if (columns == null || columns.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    StringBuilder sqlBuilder = new StringBuilder();
+    sqlBuilder.append("select ");
+    sqlBuilder.append(String.join(", ", columns));
+
+    return new SqlQueryBuilder(sqlBuilder);
+  }
+
+  /**
+   * Appends a FROM clause to a query.
+   *
+   * @param tables The table names to select from.
+   * @return The {@link SqlQueryBuilder} instance.
+   */
+  public SqlQueryBuilder from(String... tables) {
+    if (tables == null || tables.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    sqlBuilder.append(" from ");
+    sqlBuilder.append(String.join(", ", tables));

Review comment:
       for now, we are keeping it simple to fetching a single table based on a some checkpoint column. This was discussed in the RFC, to keep this scoped down and have something working end-to-end. 

##########
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);

Review comment:
       +1 to throw an exception. 

##########
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:
       is there a way to simplify these checks and fetch a default value for the storage level? 

##########
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");

Review comment:
       this could remain `info` IMO. its not really a unexpected scenario.

##########
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:
       is this import needed ? I don't think we depend on jetbrains anywhere or am I wrong?

##########
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);

Review comment:
       +1 

##########
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;

Review comment:
       can we use `property.toString()` instead? 

##########
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:
       can we name these `hoodie.deltastreamer.jdbc....` ? consistent with other source props

##########
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;

Review comment:
       +1 lets not please return null :) 

##########
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:
       +1 . we can add some validation of column types used as checkpoint. 

##########
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())) {

Review comment:
       Not sure if this is needed right @nsivabalan ?  This is no different from Kafka source not having any offsets checkpointed already. We can simply pull the entire table.

##########
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.");

Review comment:
       +1 for controlling this via a config.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-856841396


   @codope : Very few comments left to be addressed. once addressed we should be good to land this. 


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r654276684



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;

Review comment:
       Makes sense. Will update the PR.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r651418316



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;

Review comment:
       not sure on this. we might upsert duplicate records to hudi if checkpoint alone keeps failing. Let me think about it mor and also check what other sources do on similar lines. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627846007



##########
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

Review comment:
       can you help me understand how these upper bounds/lowerbounds will be used. We have sourceLimit and auto checkpointing mechanism right. so not sure how this will pan out along with that. 
   Not asking to get it fixed in this PR. just to clarify things.

##########
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())) {

Review comment:
       if a user wants to start an incremental fetch for the first time, is he/she expected to set checkpoint string? if not, this falls back to full scan right. Did you think if we can automatically choose default vals depending on column types. 

##########
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:
       not sure if this will be an overkill. Do we need to validate the incremental column for its datatype. for eg, what incase a byte[] column was chosen as incremental column. Also, another validation is to check if the column exists in the table. 

##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJdbcSource.java
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.storage.StorageLevel;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+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.sql.Statement;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests {@link JdbcSource}.
+ */
+public class TestJdbcSource extends UtilitiesTestBase {
+
+  private static final Logger LOG = LogManager.getLogger(TestJdbcSource.class);
+  private static final TypedProperties PROPS = new TypedProperties();
+
+  private static Connection connection;
+  private static final HoodieTestDataGenerator DATA_GENERATOR = new HoodieTestDataGenerator();
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    PROPS.setProperty("hoodie.datasource.jdbc.url", "jdbc:h2:mem:test_mem");
+    PROPS.setProperty("hoodie.datasource.jdbc.driver.class", "org.h2.Driver");
+    PROPS.setProperty("hoodie.datasource.jdbc.user", "test");
+    PROPS.setProperty("hoodie.datasource.jdbc.password", "jdbc");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.name", "triprec");
+    connection = DriverManager.getConnection("jdbc:h2:mem:test_mem", "test", "jdbc");
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+    close(connection);
+  }
+
+  @Test
+  public void testSingleCommit() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      int numRecords = 100;
+      String commitTime = "000";
+
+      // Insert 100 records with commit time
+      clearAndInsert(commitTime, numRecords);
+
+      // Validate if we have specified records in db
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), numRecords);
+      assertEquals(numRecords, rowDataset.count());
+    } catch (SQLException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testInsertAndUpdate() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      final String commitTime = "000";
+      final int numRecords = 100;
+
+      // Add 100 records. Update half of them with commit time "007".
+      update("007",
+          clearAndInsert(commitTime, numRecords)
+              .stream()
+              .limit(50)
+              .collect(Collectors.toList())
+      );
+      // Check if database has 100 records
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 100);
+      assertEquals(100, rowDataset.count());
+
+      Dataset<Row> firstCommit = rowDataset.where("commit_time=000");
+      assertEquals(50, firstCommit.count());
+
+      Dataset<Row> secondCommit = rowDataset.where("commit_time=007");
+      assertEquals(50, secondCommit.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testTwoCommits() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Add 10 records with commit time 001
+      insert("001", 5);
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+      assertEquals(5, rowDataset.where("commit_time=001").count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Start second commit and check if all records are pulled
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));

Review comment:
       was this added while testing locally. can we remove if not required.

##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJdbcSource.java
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.storage.StorageLevel;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+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.sql.Statement;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests {@link JdbcSource}.
+ */
+public class TestJdbcSource extends UtilitiesTestBase {
+
+  private static final Logger LOG = LogManager.getLogger(TestJdbcSource.class);
+  private static final TypedProperties PROPS = new TypedProperties();
+
+  private static Connection connection;
+  private static final HoodieTestDataGenerator DATA_GENERATOR = new HoodieTestDataGenerator();
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    PROPS.setProperty("hoodie.datasource.jdbc.url", "jdbc:h2:mem:test_mem");
+    PROPS.setProperty("hoodie.datasource.jdbc.driver.class", "org.h2.Driver");
+    PROPS.setProperty("hoodie.datasource.jdbc.user", "test");
+    PROPS.setProperty("hoodie.datasource.jdbc.password", "jdbc");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.name", "triprec");
+    connection = DriverManager.getConnection("jdbc:h2:mem:test_mem", "test", "jdbc");
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+    close(connection);
+  }
+
+  @Test
+  public void testSingleCommit() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      int numRecords = 100;
+      String commitTime = "000";
+
+      // Insert 100 records with commit time
+      clearAndInsert(commitTime, numRecords);
+
+      // Validate if we have specified records in db
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), numRecords);
+      assertEquals(numRecords, rowDataset.count());
+    } catch (SQLException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testInsertAndUpdate() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      final String commitTime = "000";
+      final int numRecords = 100;
+
+      // Add 100 records. Update half of them with commit time "007".
+      update("007",
+          clearAndInsert(commitTime, numRecords)
+              .stream()
+              .limit(50)
+              .collect(Collectors.toList())
+      );
+      // Check if database has 100 records
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 100);
+      assertEquals(100, rowDataset.count());
+
+      Dataset<Row> firstCommit = rowDataset.where("commit_time=000");
+      assertEquals(50, firstCommit.count());
+
+      Dataset<Row> secondCommit = rowDataset.where("commit_time=007");
+      assertEquals(50, secondCommit.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testTwoCommits() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Add 10 records with commit time 001
+      insert("001", 5);
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+      assertEquals(5, rowDataset.where("commit_time=001").count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Start second commit and check if all records are pulled
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));
+
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      // Start incremental scan
+      Dataset<Row> rowDataset1 = runSource(Option.of(max), 10);
+      assertEquals(10, rowDataset1.count());
+      assertEquals(10, rowDataset1.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchFallbackToFullFetchWhenError() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));
+
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "dummy_col");
+
+      // Start incremental scan with negative sourceLimit.
+      // This will throw an exception as limit clause does not accept negative values.
+      Dataset<Row> rowDataset1 = runSource(Option.of(max), -1);
+      assertEquals(20, rowDataset1.count());
+      assertEquals(10, rowDataset1.where("commit_time=000").count());
+      assertEquals(10, rowDataset1.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testFullFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      // Start full fetch
+      rowDataset = runSource(Option.empty(), 20);
+      assertEquals(20, rowDataset.count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+      assertEquals(10, rowDataset.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testFullFetchWithCheckpoint() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset

Review comment:
       you could move this to a private method (fetching incremental column max value)

##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJdbcSource.java
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.storage.StorageLevel;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+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.sql.Statement;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests {@link JdbcSource}.
+ */
+public class TestJdbcSource extends UtilitiesTestBase {
+
+  private static final Logger LOG = LogManager.getLogger(TestJdbcSource.class);
+  private static final TypedProperties PROPS = new TypedProperties();
+
+  private static Connection connection;
+  private static final HoodieTestDataGenerator DATA_GENERATOR = new HoodieTestDataGenerator();
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    PROPS.setProperty("hoodie.datasource.jdbc.url", "jdbc:h2:mem:test_mem");
+    PROPS.setProperty("hoodie.datasource.jdbc.driver.class", "org.h2.Driver");
+    PROPS.setProperty("hoodie.datasource.jdbc.user", "test");
+    PROPS.setProperty("hoodie.datasource.jdbc.password", "jdbc");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.name", "triprec");
+    connection = DriverManager.getConnection("jdbc:h2:mem:test_mem", "test", "jdbc");
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+    close(connection);
+  }
+
+  @Test
+  public void testSingleCommit() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      int numRecords = 100;
+      String commitTime = "000";
+
+      // Insert 100 records with commit time
+      clearAndInsert(commitTime, numRecords);
+
+      // Validate if we have specified records in db
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), numRecords);
+      assertEquals(numRecords, rowDataset.count());
+    } catch (SQLException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testInsertAndUpdate() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      final String commitTime = "000";
+      final int numRecords = 100;
+
+      // Add 100 records. Update half of them with commit time "007".
+      update("007",
+          clearAndInsert(commitTime, numRecords)
+              .stream()
+              .limit(50)
+              .collect(Collectors.toList())
+      );
+      // Check if database has 100 records
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 100);
+      assertEquals(100, rowDataset.count());
+
+      Dataset<Row> firstCommit = rowDataset.where("commit_time=000");
+      assertEquals(50, firstCommit.count());
+
+      Dataset<Row> secondCommit = rowDataset.where("commit_time=007");
+      assertEquals(50, secondCommit.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testTwoCommits() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Add 10 records with commit time 001
+      insert("001", 5);
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+      assertEquals(5, rowDataset.where("commit_time=001").count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Start second commit and check if all records are pulled
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));
+
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      // Start incremental scan
+      Dataset<Row> rowDataset1 = runSource(Option.of(max), 10);
+      assertEquals(10, rowDataset1.count());
+      assertEquals(10, rowDataset1.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchFallbackToFullFetchWhenError() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));
+
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "dummy_col");
+
+      // Start incremental scan with negative sourceLimit.
+      // This will throw an exception as limit clause does not accept negative values.
+      Dataset<Row> rowDataset1 = runSource(Option.of(max), -1);
+      assertEquals(20, rowDataset1.count());
+      assertEquals(10, rowDataset1.where("commit_time=000").count());
+      assertEquals(10, rowDataset1.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testFullFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");

Review comment:
       you could make a private method and reuse across diff tests. for eg, you could
   make this test and testIncrementalFetchWithCommitTime could reuse some common code. Similarly try to see if you can reuse code wherever possible even in tests. In fact, tests are critical as it would make adding more tests easier instead of monolith tests. 

##########
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;

Review comment:
       would prefer throw on any unexpected exceptions. (log and throw)

##########
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.");

Review comment:
       Open to discussion. wondering if we should just throw the exception and let user resubmit the job w/ full scan if required. These tables could be large. not sure if we can automatically falling back to full scan when incremental config is set. Or at least introduce a config "full_scan_on_exception_with_incremental" or something. 

##########
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";
+
+    private static final String URL_PROP = "url";
+
+    /**
+     * {@value #USER} is the username used for JDBC connection.
+     */
+    private static final String USER = "hoodie.datasource.jdbc.user";
+
+    /**
+     * {@value #USER_PROP} used internally to build jdbc params.
+     */
+    private static final String USER_PROP = "user";
+
+    /**
+     * {@value #PASSWORD} is the password used for JDBC connection.
+     */
+    private static final String PASSWORD = "hoodie.datasource.jdbc.password";
+
+    /**
+     * {@value #PASSWORD_FILE} is the base-path for the JDBC password file.
+     */
+    private static final String PASSWORD_FILE = "hoodie.datasource.jdbc.password.file";
+
+    /**
+     * {@value #PASSWORD_PROP} used internally to build jdbc params.
+     */
+    private static final String PASSWORD_PROP = "password";
+
+    /**
+     * {@value #DRIVER_CLASS} used for JDBC connection.
+     */
+    private static final String DRIVER_CLASS = "hoodie.datasource.jdbc.driver.class";
+
+    /**
+     * {@value #DRIVER_PROP} used internally to build jdbc params.
+     */
+    private static final String DRIVER_PROP = "driver";
+
+    /**
+     * {@value #RDBMS_TABLE_NAME} RDBMS table to pull.
+     */
+    private static final String RDBMS_TABLE_NAME = "hoodie.datasource.jdbc.table.name";
+
+    /**
+     * {@value #RDBMS_TABLE_PROP} used internally for jdbc.
+     */
+    private static final String RDBMS_TABLE_PROP = "dbtable";
+
+    /**
+     * {@value #INCREMENTAL_COLUMN} if ran in incremental mode, this field will be used to pull new data incrementally.
+     */
+    private static final String INCREMENTAL_COLUMN = "hoodie.datasource.jdbc.table.incremental.column.name";
+
+    /**
+     * {@value #IS_INCREMENTAL} will the JDBC source do an incremental pull?
+     */
+    private static final String IS_INCREMENTAL = "hoodie.datasource.jdbc.incremental.pull";
+
+    /**
+     * {@value #EXTRA_OPTIONS} used to set any extra options the user specifies for jdbc.
+     */
+    private static final String EXTRA_OPTIONS = "hoodie.datasource.jdbc.extra.options.";
+
+    /**
+     * {@value #STORAGE_LEVEL} is used to control the persistence level. Default value: MEMORY_AND_DISK_SER.
+     */
+    private static final String STORAGE_LEVEL = "hoodie.datasource.jdbc.storage.level";
+  }
+}

Review comment:
       line break at EOF

##########
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);

Review comment:
       I guess we should throw exception there rather than swallowing. Whats the next step here, if one plans to run deltastreamer in continuous mode(and if exception is thrown). It will just keep on running in a loop w/o doing any work. 
   

##########
File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJdbcSource.java
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.storage.StorageLevel;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+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.sql.Statement;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests {@link JdbcSource}.
+ */
+public class TestJdbcSource extends UtilitiesTestBase {
+
+  private static final Logger LOG = LogManager.getLogger(TestJdbcSource.class);
+  private static final TypedProperties PROPS = new TypedProperties();
+
+  private static Connection connection;
+  private static final HoodieTestDataGenerator DATA_GENERATOR = new HoodieTestDataGenerator();
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    PROPS.setProperty("hoodie.datasource.jdbc.url", "jdbc:h2:mem:test_mem");
+    PROPS.setProperty("hoodie.datasource.jdbc.driver.class", "org.h2.Driver");
+    PROPS.setProperty("hoodie.datasource.jdbc.user", "test");
+    PROPS.setProperty("hoodie.datasource.jdbc.password", "jdbc");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.name", "triprec");
+    connection = DriverManager.getConnection("jdbc:h2:mem:test_mem", "test", "jdbc");
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+    close(connection);
+  }
+
+  @Test
+  public void testSingleCommit() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      int numRecords = 100;
+      String commitTime = "000";
+
+      // Insert 100 records with commit time
+      clearAndInsert(commitTime, numRecords);
+
+      // Validate if we have specified records in db
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), numRecords);
+      assertEquals(numRecords, rowDataset.count());
+    } catch (SQLException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testInsertAndUpdate() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      final String commitTime = "000";
+      final int numRecords = 100;
+
+      // Add 100 records. Update half of them with commit time "007".
+      update("007",
+          clearAndInsert(commitTime, numRecords)
+              .stream()
+              .limit(50)
+              .collect(Collectors.toList())
+      );
+      // Check if database has 100 records
+      assertEquals(numRecords, count());
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 100);
+      assertEquals(100, rowDataset.count());
+
+      Dataset<Row> firstCommit = rowDataset.where("commit_time=000");
+      assertEquals(50, firstCommit.count());
+
+      Dataset<Row> secondCommit = rowDataset.where("commit_time=007");
+      assertEquals(50, secondCommit.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testTwoCommits() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Add 10 records with commit time 001
+      insert("001", 5);
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+      assertEquals(5, rowDataset.where("commit_time=001").count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+
+      // Start second commit and check if all records are pulled
+      rowDataset = runSource(Option.empty(), 15);
+      assertEquals(15, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));
+
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      // Start incremental scan
+      Dataset<Row> rowDataset1 = runSource(Option.of(max), 10);
+      assertEquals(10, rowDataset1.count());
+      assertEquals(10, rowDataset1.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testIncrementalFetchFallbackToFullFetchWhenError() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "true");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));
+
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "dummy_col");
+
+      // Start incremental scan with negative sourceLimit.
+      // This will throw an exception as limit clause does not accept negative values.
+      Dataset<Row> rowDataset1 = runSource(Option.of(max), -1);
+      assertEquals(20, rowDataset1.count());
+      assertEquals(10, rowDataset1.where("commit_time=000").count());
+      assertEquals(10, rowDataset1.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testFullFetchWithCommitTime() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      // Start full fetch
+      rowDataset = runSource(Option.empty(), 20);
+      assertEquals(20, rowDataset.count());
+      assertEquals(10, rowDataset.where("commit_time=000").count());
+      assertEquals(10, rowDataset.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testFullFetchWithCheckpoint() {
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+    PROPS.setProperty("hoodie.datasource.jdbc.table.incremental.column.name", "last_insert");
+
+    try {
+      // Add 10 records with commit time "000"
+      clearAndInsert("000", 10);
+
+      // Start JdbcSource
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+
+      // Get max of incremental column
+      Column incrementalColumn = rowDataset
+          .col(PROPS.getString("hoodie.datasource.jdbc.table.incremental.column.name"));
+      final String max = rowDataset.agg(functions.max(incrementalColumn).cast(DataTypes.StringType)).first()
+          .getString(0);
+      LOG.info(String.format("Incremental max value: %s", max));
+
+      // Add 10 records with commit time "001"
+      insert("001", 10);
+
+      // Start incremental scan
+      rowDataset = runSource(Option.of(max), 10);
+      assertEquals(10, rowDataset.count());
+      assertEquals(10, rowDataset.where("commit_time=001").count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testSourceWithPasswordOnFs() {
+    try {
+      // Write secret string to fs in a file
+      writeSecretToFs();
+      // Remove secret string from props
+      PROPS.remove("hoodie.datasource.jdbc.password");
+      // Set property to read secret from fs file
+      PROPS.setProperty("hoodie.datasource.jdbc.password.file", "file:///tmp/hudi/config/secret");
+      PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+      // Add 10 records with commit time 000
+      clearAndInsert("000", 10);
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testSourceWithNoPasswordThrowsException() {
+    assertThrows(NoSuchElementException.class, () -> {
+      // Write secret string to fs in a file
+      writeSecretToFs();
+      // Remove secret string from props
+      PROPS.remove("hoodie.datasource.jdbc.password");
+      PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+      // Add 10 records with commit time 000
+      clearAndInsert("000", 10);
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+    });
+  }
+
+  @Test
+  public void testSourceWithExtraOptions() {
+    PROPS.setProperty("hoodie.datasource.jdbc.extra.options.fetchsize", "10");
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+    try {
+      // Add 20 records with commit time 000
+      clearAndInsert("000", 20);
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(20, rowDataset.count());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testSourceWithStorageLevel() {
+    PROPS.setProperty("hoodie.datasource.jdbc.storage.level", "NONE");
+    PROPS.setProperty("hoodie.datasource.jdbc.incremental.pull", "false");
+    try {
+      // Add 10 records with commit time 000
+      clearAndInsert("000", 10);
+      Dataset<Row> rowDataset = runSource(Option.empty(), 10);
+      assertEquals(10, rowDataset.count());
+      LOG.error("Storage Level: " + rowDataset.storageLevel().toString());
+      assertEquals(StorageLevel.NONE(), rowDataset.storageLevel());
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+
+  private void writeSecretToFs() throws IOException {
+    FileSystem fs = FileSystem.get(new Configuration());
+    FSDataOutputStream outputStream = fs.create(new Path("file:///tmp/hudi/config/secret"));
+    outputStream.writeBytes("jdbc");
+    outputStream.close();
+  }
+
+  private static List<HoodieRecord> clearAndInsert(String commitTime, int numRecords)
+      throws SQLException {
+    execute("DROP TABLE triprec", "Table does not exists");
+    execute("CREATE TABLE triprec ("
+        + "id INT NOT NULL AUTO_INCREMENT(1, 1),"
+        + "commit_time VARCHAR(50),"
+        + "row_key VARCHAR(50),"
+        + "rider VARCHAR(50),"
+        + "driver VARCHAR(50),"
+        + "begin_lat DOUBLE PRECISION,"
+        + "begin_lon DOUBLE PRECISION,"
+        + "end_lat DOUBLE PRECISION,"
+        + "end_lon DOUBLE PRECISION,"
+        + "fare DOUBLE PRECISION,"
+        + "last_insert TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP)", "Table already exists");
+
+    return insert(commitTime, numRecords);
+  }
+
+  private static List<HoodieRecord> insert(String commitTime, int numRecords) throws SQLException {
+    PreparedStatement insertStatement =
+        connection.prepareStatement("INSERT INTO triprec ("
+            + "commit_time,"
+            + "row_key,"
+            + "rider,"
+            + "driver,"
+            + "begin_lat,"
+            + "begin_lon,"
+            + "end_lat,"
+            + "end_lon,"
+            + "fare) "
+            + "values(?,?,?,?,?,?,?,?,?)");
+    List<HoodieRecord> hoodieRecords = DATA_GENERATOR.generateInserts(commitTime, numRecords);
+
+    hoodieRecords
+        .stream()
+        .map(r -> {
+          try {
+            return ((GenericRecord) r.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, PROPS).get());
+          } catch (IOException e) {
+            return null;
+          }
+        })
+        .filter(Objects::nonNull)
+        .forEach(record -> {
+          try {
+            insertStatement.setString(1, commitTime);
+            insertStatement.setString(2, record.get("_row_key").toString());
+            insertStatement.setString(3, record.get("rider").toString());
+            insertStatement.setString(4, record.get("driver").toString());
+            insertStatement.setDouble(5, Double.parseDouble(record.get("begin_lat").toString()));
+            insertStatement.setDouble(6, Double.parseDouble(record.get("begin_lon").toString()));
+            insertStatement.setDouble(7, Double.parseDouble(record.get("end_lat").toString()));
+            insertStatement.setDouble(8, Double.parseDouble(record.get("end_lon").toString()));
+            insertStatement.setDouble(9, Double.parseDouble(((GenericRecord) record.get("fare")).get("amount").toString()));
+            insertStatement.addBatch();
+          } catch (SQLException e) {
+            LOG.warn(e.getMessage());
+          }
+        });
+    insertStatement.executeBatch();
+    close(insertStatement);
+    return hoodieRecords;
+  }
+
+  private static List<HoodieRecord> update(String commitTime, List<HoodieRecord> inserts) throws SQLException, IOException {
+    PreparedStatement updateStatement =
+        connection.prepareStatement("UPDATE triprec set commit_time=?,"
+            + "row_key=?,"
+            + "rider=?,"
+            + "driver=?,"
+            + "begin_lat=?,"
+            + "begin_lon=?,"
+            + "end_lat=?,"
+            + "end_lon=?,"
+            + "fare=?"
+            + "where row_key=?");
+
+    List<HoodieRecord> updateRecords = DATA_GENERATOR.generateUpdates(commitTime, inserts);
+    updateRecords.stream().map(m -> {
+      try {
+        return m.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, PROPS).get();
+      } catch (IOException e) {
+        return null;
+      }
+    }).filter(Objects::nonNull)
+        .map(r -> ((GenericRecord) r))
+        .sequential()
+        .forEach(r -> {
+          try {
+            updateStatement.setString(1, commitTime);
+            updateStatement.setString(2, r.get("_row_key").toString());
+            updateStatement.setString(3, r.get("rider").toString());
+            updateStatement.setString(4, r.get("driver").toString());
+            updateStatement.setDouble(5, Double.parseDouble(r.get("begin_lat").toString()));
+            updateStatement.setDouble(6, Double.parseDouble(r.get("begin_lon").toString()));
+            updateStatement.setDouble(7, Double.parseDouble(r.get("end_lat").toString()));
+            updateStatement.setDouble(8, Double.parseDouble(r.get("end_lon").toString()));
+            updateStatement.setDouble(9, Double.parseDouble(((GenericRecord) r.get("fare")).get("amount").toString()));
+            updateStatement.setString(10, r.get("_row_key").toString());
+            updateStatement.addBatch();
+          } catch (SQLException e) {
+            LOG.warn(e.getMessage());
+          }
+        });
+    updateStatement.executeBatch();
+    close(updateStatement);
+    return updateRecords;
+  }
+
+  private static void execute(String query, String message) {
+    try (Statement statement = connection.createStatement()) {
+      statement.executeUpdate(query);
+    } catch (SQLException e) {
+      LOG.error(message);

Review comment:
       shouldn't the test fail here. 




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r650843643



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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 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.deltastreamer.jdbc.extra.options. will be added.
+   * <p>
+   * Example: hoodie.deltastreamer.jdbc.extra.options.fetchSize=100
+   * hoodie.deltastreamer.jdbc.extra.options.upperBound=1
+   * hoodie.deltastreamer.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 = property.toString();
+      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) throws HoodieException {
+    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);
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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(sourceLimit);
+    }
+    dataset.persist(StorageLevel.fromString(props.getString(Config.STORAGE_LEVEL, "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, lastCkptStr));
+    dataset.unpersist();
+    return pair;
+  }
+
+  /**
+   * Does an incremental scan with PPQ query prepared on the bases of previous checkpoint.
+   *
+   * @param lastCheckpoint Last checkpoint.
+   *                       Note that the records fetched will be exclusive of the last checkpoint (i.e. incremental column value > lastCheckpoint).
+   * @return The {@link Dataset} after incremental fetch from RDBMS.
+   */
+  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.info("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);
+      if (props.containsKey(Config.FALLBACK_TO_FULL_FETCH) && props.getBoolean(Config.FALLBACK_TO_FULL_FETCH)) {
+        LOG.warn("Falling back to full scan.");
+        return fullFetch(sourceLimit);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Does a full scan on the RDBMS data source.
+   *
+   * @return The {@link Dataset} after running full scan.
+   */
+  private Dataset<Row> fullFetch(long sourceLimit) {
+    final String ppdQuery = "(%s) rdbms_table";
+    final SqlQueryBuilder queryBuilder = SqlQueryBuilder.select("*")
+        .from(props.getString(Config.RDBMS_TABLE_NAME));
+    if (sourceLimit > 0) {
+      URI jdbcURI = URI.create(props.getString(Config.URL).substring(URI_JDBC_PREFIX.length()));
+      if (DB_LIMIT_CLAUSE.contains(jdbcURI.getScheme())) {
+        if (props.containsKey(Config.INCREMENTAL_COLUMN)) {
+          queryBuilder.orderBy(props.getString(Config.INCREMENTAL_COLUMN)).limit(sourceLimit);
+        } else {
+          queryBuilder.limit(sourceLimit);
+        }
+      }
+    }
+    String query = String.format(ppdQuery, queryBuilder.toString());
+    return validatePropsAndGetDataFrameReader(sparkSession, props).option(Config.RDBMS_TABLE_PROP, query).load();
+  }
+
+  private String checkpoint(Dataset<Row> rowDataset, boolean isIncremental, Option<String> lastCkptStr) {
+    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));
+        if (max != null) {
+          return max;
+        }
+        return lastCkptStr.isPresent() && !StringUtils.isNullOrEmpty(lastCkptStr.get()) ? lastCkptStr.get() : StringUtils.EMPTY_STRING;
+      } else {
+        return StringUtils.EMPTY_STRING;
+      }
+    } catch (Exception e) {
+      return StringUtils.EMPTY_STRING;

Review comment:
       In the latest revision of the PR, I have updated to return the last known checkpoint (empty if not known).




-- 
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:
users@infra.apache.org



[GitHub] [hudi] codecov-commenter commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-832697076


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2915](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d78b858) into [master](https://codecov.io/gh/apache/hudi/commit/1db904a12ed295e08ded14bb5677dc02d76887a2?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1db904a) will **decrease** coverage by `45.42%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2915/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2915       +/-   ##
   ============================================
   - Coverage     54.22%   8.79%   -45.43%     
   + Complexity     3809      48     -3761     
   ============================================
     Files           488      56      -432     
     Lines         23574    2126    -21448     
     Branches       2510     247     -2263     
   ============================================
   - Hits          12783     187    -12596     
   + Misses         9637    1926     -7711     
   + Partials       1154      13     -1141     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `8.79% <0.00%> (-60.74%)` | `48.00 <0.00> (-326.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2915?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ava/org/apache/hudi/utilities/SqlQueryBuilder.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1NxbFF1ZXJ5QnVpbGRlci5qYXZh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [.../org/apache/hudi/utilities/sources/JdbcSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSmRiY1NvdXJjZS5qYXZh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | ... and [469 more](https://codecov.io/gh/apache/hudi/pull/2915/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan merged pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan merged pull request #2915:
URL: https://github.com/apache/hudi/pull/2915


   


-- 
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:
users@infra.apache.org



[GitHub] [hudi] nsivabalan commented on pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#issuecomment-850821638


   @codope : all feedback addressed? can we review. or do ping us here once its ready for review.


-- 
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:
users@infra.apache.org



[GitHub] [hudi] codope commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
codope commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r633695998



##########
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.




-- 
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:
users@infra.apache.org



[GitHub] [hudi] leesf commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627417386



##########
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");

Review comment:
       warn would be better?




-- 
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:
users@infra.apache.org



[GitHub] [hudi] leesf commented on a change in pull request #2915: [HUDI-251] Adds JDBC source support for DeltaStreamer

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #2915:
URL: https://github.com/apache/hudi/pull/2915#discussion_r627417046



##########
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);

Review comment:
       would be warn level since the exception does not kill the app?




-- 
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:
users@infra.apache.org