taherk77 commented on a change in pull request #917: [HUDI-251] JDBC 
incremental load to HUDI with DeltaStreamer
URL: https://github.com/apache/incubator-hudi/pull/917#discussion_r329909616
 
 

 ##########
 File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JDBCSource.java
 ##########
 @@ -0,0 +1,233 @@
+package org.apache.hudi.utilities.sources;
+
+import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
+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.hudi.DataSourceUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+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.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JDBCSource extends RowSource {
+
+  private static Logger LOG = LoggerFactory.getLogger(JDBCSource.class);
+
+  private final String ppdQuery = "(select * from %s where %s >= \" %s \") 
rdbms_table";
+
+
+  public JDBCSource(TypedProperties props, JavaSparkContext sparkContext, 
SparkSession sparkSession,
+      SchemaProvider schemaProvider) {
+    super(props, sparkContext, sparkSession, schemaProvider);
+  }
+
+  private static DataFrameReader validatePropsAndGetDataFrameReader(final 
SparkSession session,
+      final TypedProperties properties)
+      throws HoodieException {
+    FSDataInputStream passwordFileStream = null;
+    try {
+      DataFrameReader 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)) {
+        if (properties.containsKey(Config.PASSWORD_FILE)) {
+          if 
(!StringUtils.isNullOrEmpty(properties.getString(Config.PASSWORD_FILE))) {
+            LOG.info("Reading password for password file {}", 
properties.getString(Config.PASSWORD_FILE));
+            FileSystem fileSystem = FileSystem.get(new Configuration());
+            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("%s property cannot be null or empty", 
Config.PASSWORD_FILE));
+          }
+        } else {
+          throw new IllegalArgumentException(String.format("JDBCSource needs 
either a %s or %s to connect to RDBMS "
+              + "datasource", Config.PASSWORD_FILE, Config.PASSWORD));
+        }
+      } else if 
(!StringUtils.isNullOrEmpty(properties.getString(Config.PASSWORD))) {
+        dataFrameReader = dataFrameReader.option(Config.PASSWORD_PROP, 
properties.getString(Config.PASSWORD));
+      } else {
+        throw new IllegalArgumentException(String.format("%s cannot be null or 
empty. ", Config.PASSWORD));
+      }
+      if (properties.containsKey(Config.EXTRA_OPTIONS)) {
 
 Review comment:
   The extra.options is for adding some properties for jdbc pulls. so lets say 
use writes the below spark code.
   
   ```val df1 = spark.read.format("jdbc")
         .option("url", jdbcUrl)
         .option("dbtable", "datetimePartitionTest")
         .option("partitionColumn", "d")
         .option("lowerBound", "2018-07-06")
         .option("upperBound", "2018-07-20")
         .option("numPartitions", 3)
         .option("timestampFormat", "MM/dd/yyyy h:mm:ss a")
         .option("oracle.jdbc.mapDateToTimestamp", "false")
         .option("sessionInitStatement", "ALTER SESSION SET NLS_DATE_FORMAT = 
'YYYY-MM-DD'")
         .load()
   ```
   
    Now how do we let the user express the same with a properties file? We use 
extra.options 
   
    `hoodie.datasource.jdbc.extra.options="
   fetchsize=1000,
   timestampFormat=MM/dd/yyyy h:mm:ss a,
   oracle.jdbc.mapDateToTimestamp=false,
   sessionInitStatement="ALTER SESSION SET NLS_DATE_FORMAT = 'YYYY-MM-DD'"`

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


With regards,
Apache Git Services

Reply via email to