You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/01/17 07:06:52 UTC

[GitHub] [incubator-hudi] yihua opened a new pull request #1239: [MINOR] Abstract a test case class for DFS Source to make it extensible

yihua opened a new pull request #1239: [MINOR] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239
 
 
   ## What is the purpose of the pull request
   
   `TestDFSSource` class contains redundant code of the test logic for different `Source`s reading from DFS.  This PR abstracts a test case class for DFS Source to make it extensible.  It can be easily extended to run test for new DFS `Source`, such as `CsvDFSSource` to come in HUDI-76.
   
   ## Brief change log
   
     - Refactored the code in `TestDFSSource` to add a new inner abstract class `DFSSourceTestCase` to make the test case extensible and use the new class for existing tests.
   
   ## Verify this pull request
   
   This pull request is a code cleanup in the tests, and the modified tests are verified to finish successfully.
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] leesf commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
leesf commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575963722
 
 
   > > Looks like the auto code reformatting does not reorder the imports.
   > 
   > This is a pain atm. yes.
   > 
   > @yanghua is this good to go?
   
   Will help to review since @yanghua has some other stuff today.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575926661
 
 
   >Looks like the auto code reformatting does not reorder the imports.
   
   This is a pain atm. yes. 
   
   @yanghua  is this good to go? 

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575857464
 
 
   Thanks for the suggestions, @yanghua !
   
   So here's a list of proposed changes based on your comments:
   - Create a new class `AbstractDFSSourceTestBase` based on `DFSSourceTestCase` in the last commit
   - The common test logic still resides in `AbstractDFSSourceTestBase`
   - For each DFS Source class, extend from `AbstractDFSSourceTestBase` to add source-specific test logic
   
   Let me know if this looks good.  I'll start rearranging the code and go from there.
   
   Btw, the CSV Source PR (HUDI-76) depends on this.  It would be good if we can iterate quickly on this one to get it merged :)

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575873910
 
 
   @yanghua All the imports are fixed.  Looks like the auto code reformatting does not reorder the imports.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yanghua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#discussion_r368210766
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/AbstractDFSSourceTestBase.java
 ##########
 @@ -0,0 +1,185 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
 
 Review comment:
   Based on the latest checkstyle about **import**, the order of these import statement is not correct.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yanghua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#discussion_r368210796
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSource.java
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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 java.io.IOException;
 
 Review comment:
   ditto

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yanghua commented on issue #1239: [MINOR] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yanghua commented on issue #1239: [MINOR] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575533263
 
 
   Hi @yihua thanks for your contribution. The thought looks good. After thinking deeply. Can we define a top-level `AbstractDFSSourceTestBase` to do the abstraction? Then, we can have `TestJsonDFSSource`, `TestParquetDFSSource` and `TestCsvDFSSource` to make the test more extendable.
   
   IMHO, currently, the testing for the DFS source is not enough. We would add more test cases to test more scenes in the future. Split different test cases of sources into different classes is a good direction. WDYT?
   
   In addition, just a tip. The change may beyond the scope of the "MINOR". In the future, I suggest we can create jira to track these refactor.
   
   > a "MINOR" should be a small(minor) change that needs not to be file in the jira. We even may merge it before the Travis is green, If we are bold enough.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#discussion_r368211015
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/AbstractDFSSourceTestBase.java
 ##########
 @@ -0,0 +1,185 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
 
 Review comment:
   Got it.  will fix it.  Somehow local checkstyle does not prompt any error.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] leesf merged pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
leesf merged pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239
 
 
   

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] leesf commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#discussion_r368264818
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/AbstractDFSSourceTestBase.java
 ##########
 @@ -0,0 +1,186 @@
+/*
+ * 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.AvroConversionUtils;
+import org.apache.hudi.common.HoodieTestDataGenerator;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * An abstract test base for {@link Source} using DFS as the file system.
+ */
+public abstract class AbstractDFSSourceTestBase extends UtilitiesTestBase {
+
+  FilebasedSchemaProvider schemaProvider;
+  String dfsRoot;
+  String fileSuffix;
+  int fileCount = 1;
+  HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  /**
+   * Prepares the specific {@link Source} to test, by passing in necessary configurations.
+   *
+   * @return A {@link Source} using DFS as the file system.
+   */
+  abstract Source prepareDFSSource();
+
+  /**
+   * Writes test data, i.e., a {@link List} of {@link HoodieRecord}, to a file on DFS.
+   *
+   * @param records Test data.
+   * @param path    The path in {@link Path} of the file to write.
+   * @throws IOException
+   */
+  abstract void writeNewDataToFile(List<HoodieRecord> records, Path path) throws IOException;
+
+  /**
+   * Generates a batch of test data and writes the data to a file.  This can be called multiple times to generate multiple files.
+   *
+   * @return The {@link Path} of the file.
+   * @throws IOException
+   */
+  Path generateOneFile() throws IOException {
+    Path path = new Path(dfsRoot, fileCount + fileSuffix);
+    switch (fileCount) {
+      case 1:
+        writeNewDataToFile(dataGenerator.generateInserts("000", 100), path);
+        fileCount++;
+        return path;
+      case 2:
+        writeNewDataToFile(dataGenerator.generateInserts("001", 10000), path);
+        fileCount++;
+        return path;
+      default:
+        return null;
+    }
+  }
 
 Review comment:
   Would not get the point of this method, a bit of tricky.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yanghua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#discussion_r368210783
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonDFSSource.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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 java.io.IOException;
 
 Review comment:
   ditto

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yanghua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yanghua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575865264
 
 
   > Thanks for the suggestions, @yanghua !
   > 
   > So here's a list of proposed changes based on your comments:
   > 
   > * Create a new class `AbstractDFSSourceTestBase` based on `DFSSourceTestCase` in the last commit
   > * The common test logic still resides in `AbstractDFSSourceTestBase`
   > * For each DFS Source class, extend from `AbstractDFSSourceTestBase` to add source-specific test logic
   > 
   > Let me know if this looks good. I'll start rearranging the code and go from there.
   
   Yes, your summary is great! And let's track them with Jira issues. So go ahead.
   
   > Btw, the CSV Source PR (HUDI-76) depends on this. It would be good if we can iterate quickly on this one to get it merged :)
   
   Agree, I or @leesf will track the relevant jobs that you mentioned above.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#discussion_r368266524
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/AbstractDFSSourceTestBase.java
 ##########
 @@ -0,0 +1,186 @@
+/*
+ * 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.AvroConversionUtils;
+import org.apache.hudi.common.HoodieTestDataGenerator;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * An abstract test base for {@link Source} using DFS as the file system.
+ */
+public abstract class AbstractDFSSourceTestBase extends UtilitiesTestBase {
+
+  FilebasedSchemaProvider schemaProvider;
+  String dfsRoot;
+  String fileSuffix;
+  int fileCount = 1;
+  HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  /**
+   * Prepares the specific {@link Source} to test, by passing in necessary configurations.
+   *
+   * @return A {@link Source} using DFS as the file system.
+   */
+  abstract Source prepareDFSSource();
+
+  /**
+   * Writes test data, i.e., a {@link List} of {@link HoodieRecord}, to a file on DFS.
+   *
+   * @param records Test data.
+   * @param path    The path in {@link Path} of the file to write.
+   * @throws IOException
+   */
+  abstract void writeNewDataToFile(List<HoodieRecord> records, Path path) throws IOException;
+
+  /**
+   * Generates a batch of test data and writes the data to a file.  This can be called multiple times to generate multiple files.
+   *
+   * @return The {@link Path} of the file.
+   * @throws IOException
+   */
+  Path generateOneFile() throws IOException {
+    Path path = new Path(dfsRoot, fileCount + fileSuffix);
+    switch (fileCount) {
+      case 1:
+        writeNewDataToFile(dataGenerator.generateInserts("000", 100), path);
+        fileCount++;
+        return path;
+      case 2:
+        writeNewDataToFile(dataGenerator.generateInserts("001", 10000), path);
+        fileCount++;
+        return path;
+      default:
+        return null;
+    }
+  }
 
 Review comment:
   Each time it's called, the method generates a new file for a batch of data.  Only two batches are considered.  Any suggestions to make it better?

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575875861
 
 
   Rebasing on master ...

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575878732
 
 
   @yanghua The PR is ready for a final review.  I'll squash the commits before merging.

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#issuecomment-575872288
 
 
   @yanghua  I've updated the PR based on the change list.  JIRA ticket here: https://issues.apache.org/jira/browse/HUDI-551

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] yihua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #1239: [HUDI-551] Abstract a test case class for DFS Source to make it extensible
URL: https://github.com/apache/incubator-hudi/pull/1239#discussion_r368268103
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/AbstractDFSSourceTestBase.java
 ##########
 @@ -0,0 +1,186 @@
+/*
+ * 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.AvroConversionUtils;
+import org.apache.hudi.common.HoodieTestDataGenerator;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * An abstract test base for {@link Source} using DFS as the file system.
+ */
+public abstract class AbstractDFSSourceTestBase extends UtilitiesTestBase {
+
+  FilebasedSchemaProvider schemaProvider;
+  String dfsRoot;
+  String fileSuffix;
+  int fileCount = 1;
+  HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  /**
+   * Prepares the specific {@link Source} to test, by passing in necessary configurations.
+   *
+   * @return A {@link Source} using DFS as the file system.
+   */
+  abstract Source prepareDFSSource();
+
+  /**
+   * Writes test data, i.e., a {@link List} of {@link HoodieRecord}, to a file on DFS.
+   *
+   * @param records Test data.
+   * @param path    The path in {@link Path} of the file to write.
+   * @throws IOException
+   */
+  abstract void writeNewDataToFile(List<HoodieRecord> records, Path path) throws IOException;
+
+  /**
+   * Generates a batch of test data and writes the data to a file.  This can be called multiple times to generate multiple files.
+   *
+   * @return The {@link Path} of the file.
+   * @throws IOException
+   */
+  Path generateOneFile() throws IOException {
+    Path path = new Path(dfsRoot, fileCount + fileSuffix);
+    switch (fileCount) {
+      case 1:
+        writeNewDataToFile(dataGenerator.generateInserts("000", 100), path);
+        fileCount++;
+        return path;
+      case 2:
+        writeNewDataToFile(dataGenerator.generateInserts("001", 10000), path);
+        fileCount++;
+        return path;
+      default:
+        return null;
+    }
+  }
 
 Review comment:
   In my latest commit, as we discussed, I parameterized this method to take the file name, commit time String and the number of records to generate, to make it easier to understand and use.

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


With regards,
Apache Git Services