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_r329908038
 
 

 ##########
 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)) {
+        if 
(!StringUtils.isNullOrEmpty(properties.getString(Config.EXTRA_OPTIONS))) {
+          LOG.info("Setting {}", Config.EXTRA_OPTIONS);
+          String[] options = 
properties.getString(Config.EXTRA_OPTIONS).split(",");
+          for (String option : options) {
+            if (!StringUtils.isNullOrEmpty(option)) {
+              String[] kv = option.split("=");
+              dataFrameReader = dataFrameReader.option(kv[0], kv[1]);
+              LOG.info("{} = {} has been set for JDBC pull ", kv[0], kv[1]);
+            }
+          }
+        }
+      }
+      if (properties.getBoolean(Config.IS_INCREMENTAL)) {
+        DataSourceUtils.checkRequiredProperties(properties, 
Arrays.asList(Config.INCREMENTAL_COLUMN));
+      }
+      return dataFrameReader;
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    } finally {
+      IOUtils.closeStream(passwordFileStream);
+    }
+  }
+
+  @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));
+
+      Option<String> beginInstant =
+          lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? 
Option.empty() : lastCkptStr : Option.empty();
+      boolean isIncremental = props.getBoolean(Config.IS_INCREMENTAL);
+      if (beginInstant.equals(Option.empty())) {
+        LOG.info("No previous checkpoints found.. ");
+        Dataset<Row> rowDataset = 
validatePropsAndGetDataFrameReader(sparkSession, props).load();
+        return sendDfAndCheckpoint(rowDataset, isIncremental);
+      } else {
+        if (StringUtils.isNullOrEmpty(beginInstant.get())) {
+          LOG.warn("Previous checkpoint entry was null or empty. Falling back 
to full jdbc pull.");
+          Dataset<Row> rowDataset = 
validatePropsAndGetDataFrameReader(sparkSession, props).load();
+          return sendDfAndCheckpoint(rowDataset, isIncremental);
+        } else {
+          String query = String
+              .format(ppdQuery, props.getString(Config.RDBMS_TABLE_NAME), 
props.getString(Config.INCREMENTAL_COLUMN),
+                  beginInstant.get());
+          LOG.info("Referenced last checkpoint and prepared new predicate 
pushdown query for jdbc pull {}", query);
+          Dataset<Row> rowDataset = 
validatePropsAndGetDataFrameReader(sparkSession, props)
+              .option(props.getString(Config.RDBMS_TABLE_PROP), query).load();
+          return sendDfAndCheckpoint(rowDataset, isIncremental);
+
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Exception while running JDBCSource ", e);
+      return Pair.of(Option.empty(), null);
+    }
+  }
+
+  @NotNull
+  private Pair<Option<Dataset<Row>>, String> sendDfAndCheckpoint(Dataset<Row> 
rowDataset, boolean isIncremental) {
+    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("Sending {} with checkpoint val {} ", incrementalColumn, max);
+      return Pair.of(Option.of(rowDataset), max);
+    } else {
 
 Review comment:
   Let me put it this way for you. 
   case 1: A JDBC job can be continuous but not incremental
   case 2: A JDBC job can be continuous and incremental
   case 3: A JDBC job is not continuous but is incremental
   case 4: A JDBC job is not continuous and not incremental
   
   In case 1 and case 4 we never reference anything to/from the checkpoint and 
we pull the entire table always.
   
   In case 2 and 3 we reference the checkpoint to see the last checkpointed 
value. If the value is present, we build an incremental query or else we do the 
whole table pull and write the max of incremental column and keep doing this 
same thing again and again. This is what I think will be best.
   
   

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