1
0

[HUDI-69] Support Spark Datasource for MOR table - RDD approach (#1848)

- This PR implements Spark Datasource for MOR table in the RDD approach.
- Implemented SnapshotRelation
- Implemented HudiMergeOnReadRDD
- Implemented separate Iterator to handle merge and unmerge record reader.
- Added TestMORDataSource to verify this feature.
- Clean up test file name, add tests for mixed query type tests
 - We can now revert the change made in DefaultSource

Co-authored-by: Vinoth Chandar <vchandar@confluent.io>
This commit is contained in:
Gary Li
2020-08-07 00:28:14 -07:00
committed by GitHub
parent ab453f2623
commit 4f74a84607
22 changed files with 1317 additions and 409 deletions

View File

@@ -130,14 +130,7 @@ public class TestBootstrap extends HoodieClientTestBase {
public void setUp() throws Exception {
bootstrapBasePath = tmpFolder.toAbsolutePath().toString() + "/data";
initPath();
spark = SparkSession.builder()
.appName("Bootstrap test")
.master("local[2]")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.getOrCreate();
jsc = new JavaSparkContext(spark.sparkContext());
sqlContext = spark.sqlContext();
hadoopConf = spark.sparkContext().hadoopConfiguration();
initSparkContexts();
initTestDataGenerator();
initMetaClient();
// initialize parquet input format
@@ -146,6 +139,7 @@ public class TestBootstrap extends HoodieClientTestBase {
@AfterEach
public void tearDown() throws IOException {
cleanupSparkContexts();
cleanupClients();
cleanupTestDataGenerator();
}