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

2021-06-18 Thread GitBox


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 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 sessionThe {@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 

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

2021-06-14 Thread GitBox


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 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 sessionThe {@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 

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

2021-06-14 Thread GitBox


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




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

2021-06-14 Thread GitBox


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 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 sessionThe {@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 {
+

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

2021-06-14 Thread GitBox


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 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 sessionThe {@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 {
+

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

2021-06-14 Thread GitBox


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 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 sessionThe {@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 

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

2021-06-14 Thread GitBox


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

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

2021-05-17 Thread GitBox


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 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 sessionThe {@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 {
+

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

2021-05-17 Thread GitBox


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 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 sessionThe {@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 {
+

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

2021-05-17 Thread GitBox


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 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 sessionThe {@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 {
+

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

2021-05-17 Thread GitBox


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




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

2021-05-17 Thread GitBox


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




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

2021-05-17 Thread GitBox


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 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 sessionThe {@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 {
+