[GitHub] [hudi] vingov commented on a change in pull request #2896: [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case

2021-06-09 Thread GitBox


vingov commented on a change in pull request #2896:
URL: https://github.com/apache/hudi/pull/2896#discussion_r648020965



##
File path: 
hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestSqlSource.java
##
@@ -0,0 +1,121 @@
+/*
+ * 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.testutils.sources;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.AvroConversionUtils;
+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.utilities.deltastreamer.SourceFormatAdapter;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.sources.InputBatch;
+import org.apache.hudi.utilities.sources.SqlSource;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import java.io.IOException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Test against {@link SqlSource}.
+ */
+public class TestSqlSource extends UtilitiesTestBase {
+
+  private final boolean useFlattenedSchema = false;
+  protected FilebasedSchemaProvider schemaProvider;
+  protected HoodieTestDataGenerator dataGenerator = new 
HoodieTestDataGenerator();
+  private String dfsRoot;
+  private String fileSuffix;
+
+  @BeforeAll
+  public static void initClass() throws Exception {
+UtilitiesTestBase.initClass();
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+UtilitiesTestBase.cleanupClass();
+  }
+
+  @BeforeEach
+  public void setup() throws Exception {
+dfsRoot = dfsBasePath + "/parquetFiles";
+super.setup();
+schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), 
jsc);
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+super.teardown();
+  }
+
+  /**
+   * Generates a batch of test data and writes the data to a file and register 
a test table.
+   *
+   * @param filenameThe name of the file.
+   * @param instantTime The commit time.
+   * @param n   The number of records to generate.
+   */
+  private void generateTestTable(String filename, String instantTime, int n) 
throws IOException {
+Path path = new Path(dfsRoot, filename + fileSuffix);
+
Helpers.saveParquetToDFS(Helpers.toGenericRecords(dataGenerator.generateInserts(instantTime,
 n, useFlattenedSchema)), path);
+
sparkSession.read().parquet(dfsRoot).createOrReplaceTempView("test_sql_table");
+  }
+
+  /**
+   * Runs the test scenario of reading data from the source.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testSqlSource() throws IOException {
+UtilitiesTestBase.dfs.mkdirs(new Path(dfsRoot));
+TypedProperties props = new TypedProperties();
+props.setProperty("hoodie.deltastreamer.source.sql", "select * from 
test_sql_table");
+SqlSource sqlSource = new SqlSource(props, jsc, sparkSession, 
schemaProvider);
+SourceFormatAdapter sourceFormatAdapter = new 
SourceFormatAdapter(sqlSource);
+
+// Produce new data, extract new data
+generateTestTable("1", "001", 1);
+
+// Test fetching Avro format
+InputBatch> fetch1 =
+sourceFormatAdapter.fetchNewDataInAvroFormat(Option.empty(), 
Long.MAX_VALUE);
+
+// Test Avro to Row format
+Dataset fetch1Rows = AvroConversionUtils
+.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()),
+schemaProvider.getSourceSchema().toString(), sparkSession);
+assertEquals(1, fetch1Rows.count());
+
+// Test fetching Row format
+InputBatch> fetch1AsRows =
+sourceFormatAdapter.fetchNewDataInRowFormat(Option.empty(), 
Long.MAX_VALUE);
+assertEquals(1, fetch1AsRows.getBatch().get().count());
+  }

Review comment:
   Sure, added all the four tests.




-- 
This is 

[GitHub] [hudi] vingov commented on a change in pull request #2896: [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case

2021-06-09 Thread GitBox


vingov commented on a change in pull request #2896:
URL: https://github.com/apache/hudi/pull/2896#discussion_r648020771



##
File path: 
hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestSqlSource.java
##
@@ -0,0 +1,121 @@
+/*
+ * 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.testutils.sources;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.AvroConversionUtils;
+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.utilities.deltastreamer.SourceFormatAdapter;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.sources.InputBatch;
+import org.apache.hudi.utilities.sources.SqlSource;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import java.io.IOException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Test against {@link SqlSource}.
+ */
+public class TestSqlSource extends UtilitiesTestBase {
+
+  private final boolean useFlattenedSchema = false;
+  protected FilebasedSchemaProvider schemaProvider;
+  protected HoodieTestDataGenerator dataGenerator = new 
HoodieTestDataGenerator();
+  private String dfsRoot;
+  private String fileSuffix;
+
+  @BeforeAll
+  public static void initClass() throws Exception {
+UtilitiesTestBase.initClass();
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+UtilitiesTestBase.cleanupClass();
+  }
+
+  @BeforeEach
+  public void setup() throws Exception {
+dfsRoot = dfsBasePath + "/parquetFiles";
+super.setup();
+schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), 
jsc);
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+super.teardown();
+  }
+
+  /**
+   * Generates a batch of test data and writes the data to a file and register 
a test table.
+   *
+   * @param filenameThe name of the file.
+   * @param instantTime The commit time.
+   * @param n   The number of records to generate.
+   */
+  private void generateTestTable(String filename, String instantTime, int n) 
throws IOException {
+Path path = new Path(dfsRoot, filename + fileSuffix);
+
Helpers.saveParquetToDFS(Helpers.toGenericRecords(dataGenerator.generateInserts(instantTime,
 n, useFlattenedSchema)), path);
+
sparkSession.read().parquet(dfsRoot).createOrReplaceTempView("test_sql_table");
+  }
+
+  /**
+   * Runs the test scenario of reading data from the source.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testSqlSource() throws IOException {
+UtilitiesTestBase.dfs.mkdirs(new Path(dfsRoot));
+TypedProperties props = new TypedProperties();
+props.setProperty("hoodie.deltastreamer.source.sql", "select * from 
test_sql_table");

Review comment:
   The variable declared in the source is private, hence hardcoded it here, 
but to avoid typos made it as a final variable and used it everywhere.




-- 
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] vingov commented on a change in pull request #2896: [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case

2021-06-09 Thread GitBox


vingov commented on a change in pull request #2896:
URL: https://github.com/apache/hudi/pull/2896#discussion_r648015740



##
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SqlSource.java
##
@@ -0,0 +1,79 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.Collections;
+
+/**
+ * SQL Source that reads from any table, used mainly for backfill jobs which 
will process specific partition dates.
+ */
+public class SqlSource extends RowSource {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(SqlSource.class);
+  private final String sourceSql;
+  private final SparkSession spark;
+
+  public SqlSource(
+  TypedProperties props,
+  JavaSparkContext sparkContext,
+  SparkSession sparkSession,
+  SchemaProvider schemaProvider) {
+super(props, sparkContext, sparkSession, schemaProvider);
+DataSourceUtils.checkRequiredProperties(
+props, Collections.singletonList(SqlSource.Config.SOURCE_SQL));
+sourceSql = props.getString(SqlSource.Config.SOURCE_SQL);
+spark = sparkSession;
+  }
+
+  @Override
+  protected Pair>, String> fetchNextBatch(
+  Option lastCkptStr, long sourceLimit) {
+LOG.warn(sourceSql);
+Dataset source = spark.sql(sourceSql);
+LOG.warn(source.showString(10, 0, true));
+// Remove Hoodie meta columns except partition path from input source.
+Dataset src =
+source.drop(
+HoodieRecord.HOODIE_META_COLUMNS.stream()
+.filter(x -> 
!x.equals(HoodieRecord.PARTITION_PATH_METADATA_FIELD))
+.toArray(String[]::new));
+return Pair.of(Option.of(src), null);
+  }
+
+  /**
+   * Configs supported.
+   */
+  private static class Config {
+
+private static final String SOURCE_SQL = "hoodie.deltastreamer.source.sql";

Review comment:
   got it, updated the same.




-- 
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] vingov commented on a change in pull request #2896: [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case

2021-06-09 Thread GitBox


vingov commented on a change in pull request #2896:
URL: https://github.com/apache/hudi/pull/2896#discussion_r648015475



##
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SqlSource.java
##
@@ -0,0 +1,79 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.Collections;
+
+/**
+ * SQL Source that reads from any table, used mainly for backfill jobs which 
will process specific partition dates.
+ */
+public class SqlSource extends RowSource {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(SqlSource.class);
+  private final String sourceSql;
+  private final SparkSession spark;
+
+  public SqlSource(
+  TypedProperties props,
+  JavaSparkContext sparkContext,
+  SparkSession sparkSession,
+  SchemaProvider schemaProvider) {
+super(props, sparkContext, sparkSession, schemaProvider);
+DataSourceUtils.checkRequiredProperties(
+props, Collections.singletonList(SqlSource.Config.SOURCE_SQL));
+sourceSql = props.getString(SqlSource.Config.SOURCE_SQL);
+spark = sparkSession;
+  }
+
+  @Override
+  protected Pair>, String> fetchNextBatch(
+  Option lastCkptStr, long sourceLimit) {
+LOG.warn(sourceSql);
+Dataset source = spark.sql(sourceSql);
+LOG.warn(source.showString(10, 0, true));
+// Remove Hoodie meta columns except partition path from input source.
+Dataset src =
+source.drop(

Review comment:
   It should be mostly hudi, but you are right, it could be a non-hudi 
table as well, added checks to filter only if its a hudi table.




-- 
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] vingov commented on a change in pull request #2896: [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case

2021-06-09 Thread GitBox


vingov commented on a change in pull request #2896:
URL: https://github.com/apache/hudi/pull/2896#discussion_r648014788



##
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SqlSource.java
##
@@ -0,0 +1,79 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.Collections;
+
+/**
+ * SQL Source that reads from any table, used mainly for backfill jobs which 
will process specific partition dates.
+ */
+public class SqlSource extends RowSource {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(SqlSource.class);
+  private final String sourceSql;
+  private final SparkSession spark;
+
+  public SqlSource(
+  TypedProperties props,
+  JavaSparkContext sparkContext,
+  SparkSession sparkSession,
+  SchemaProvider schemaProvider) {
+super(props, sparkContext, sparkSession, schemaProvider);
+DataSourceUtils.checkRequiredProperties(
+props, Collections.singletonList(SqlSource.Config.SOURCE_SQL));
+sourceSql = props.getString(SqlSource.Config.SOURCE_SQL);
+spark = sparkSession;
+  }
+
+  @Override
+  protected Pair>, String> fetchNextBatch(
+  Option lastCkptStr, long sourceLimit) {
+LOG.warn(sourceSql);
+Dataset source = spark.sql(sourceSql);
+LOG.warn(source.showString(10, 0, true));
+// Remove Hoodie meta columns except partition path from input source.
+Dataset src =
+source.drop(
+HoodieRecord.HOODIE_META_COLUMNS.stream()
+.filter(x -> 
!x.equals(HoodieRecord.PARTITION_PATH_METADATA_FIELD))
+.toArray(String[]::new));
+return Pair.of(Option.of(src), null);

Review comment:
   Yes, this is more of a one time sync, clarified the same in the java 
docs.




-- 
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] vingov commented on a change in pull request #2896: [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case

2021-06-09 Thread GitBox


vingov commented on a change in pull request #2896:
URL: https://github.com/apache/hudi/pull/2896#discussion_r648014435



##
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SqlSource.java
##
@@ -0,0 +1,79 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.Collections;
+
+/**
+ * SQL Source that reads from any table, used mainly for backfill jobs which 
will process specific partition dates.
+ */
+public class SqlSource extends RowSource {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(SqlSource.class);
+  private final String sourceSql;
+  private final SparkSession spark;
+
+  public SqlSource(
+  TypedProperties props,
+  JavaSparkContext sparkContext,
+  SparkSession sparkSession,
+  SchemaProvider schemaProvider) {
+super(props, sparkContext, sparkSession, schemaProvider);
+DataSourceUtils.checkRequiredProperties(
+props, Collections.singletonList(SqlSource.Config.SOURCE_SQL));
+sourceSql = props.getString(SqlSource.Config.SOURCE_SQL);
+spark = sparkSession;
+  }
+
+  @Override
+  protected Pair>, String> fetchNextBatch(
+  Option lastCkptStr, long sourceLimit) {
+LOG.warn(sourceSql);
+Dataset source = spark.sql(sourceSql);
+LOG.warn(source.showString(10, 0, true));

Review comment:
   nope, made it debug.




-- 
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] vingov commented on a change in pull request #2896: [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case

2021-06-09 Thread GitBox


vingov commented on a change in pull request #2896:
URL: https://github.com/apache/hudi/pull/2896#discussion_r648014332



##
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SqlSource.java
##
@@ -0,0 +1,79 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.Collections;
+
+/**
+ * SQL Source that reads from any table, used mainly for backfill jobs which 
will process specific partition dates.
+ */
+public class SqlSource extends RowSource {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(SqlSource.class);
+  private final String sourceSql;
+  private final SparkSession spark;
+
+  public SqlSource(
+  TypedProperties props,
+  JavaSparkContext sparkContext,
+  SparkSession sparkSession,
+  SchemaProvider schemaProvider) {
+super(props, sparkContext, sparkSession, schemaProvider);
+DataSourceUtils.checkRequiredProperties(
+props, Collections.singletonList(SqlSource.Config.SOURCE_SQL));
+sourceSql = props.getString(SqlSource.Config.SOURCE_SQL);
+spark = sparkSession;
+  }
+
+  @Override
+  protected Pair>, String> fetchNextBatch(
+  Option lastCkptStr, long sourceLimit) {
+LOG.warn(sourceSql);

Review comment:
   this is for debugging, made it debug.




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