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:
[email protected]