[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:
@@ -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();
|
||||
}
|
||||
|
||||
@@ -28,7 +28,7 @@ import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
|
||||
class TestHudiSparkUtils {
|
||||
class TestHoodieSparkUtils {
|
||||
|
||||
@Test
|
||||
def testGlobPaths(@TempDir tempDir: File): Unit = {
|
||||
@@ -48,29 +48,29 @@ class TestHudiSparkUtils {
|
||||
files.foreach(file => new File(file.toUri).createNewFile())
|
||||
|
||||
var paths = Seq(tempDir.getAbsolutePath + "/*")
|
||||
var globbedPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
var globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(folders.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/*/*")
|
||||
globbedPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(files.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder1/*")
|
||||
globbedPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(Seq(files(0), files(1)).sortWith(_.toString < _.toString),
|
||||
globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder2/*")
|
||||
globbedPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(Seq(files(2), files(3)).sortWith(_.toString < _.toString),
|
||||
globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder1/*", tempDir.getAbsolutePath + "/folder2/*")
|
||||
globbedPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(files.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
}
|
||||
@@ -98,8 +98,9 @@ class TestHudiSparkUtils {
|
||||
folders.foreach(folder => new File(folder.toUri).mkdir())
|
||||
files.foreach(file => new File(file.toUri).createNewFile())
|
||||
|
||||
val index = HudiSparkUtils.createInMemoryFileIndex(spark, Seq(folders(0), folders(1)))
|
||||
val index = HoodieSparkUtils.createInMemoryFileIndex(spark, Seq(folders(0), folders(1)))
|
||||
val indexedFilePaths = index.allFiles().map(fs => fs.getPath)
|
||||
assertEquals(files.sortWith(_.toString < _.toString), indexedFilePaths.sortWith(_.toString < _.toString))
|
||||
spark.stop()
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,197 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for COW table.
|
||||
*/
|
||||
class TestCOWDataSource extends HoodieClientTestBase {
|
||||
private val log = LogManager.getLogger(getClass)
|
||||
var spark: SparkSession = null
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test def testShortNameStorage() {
|
||||
// Insert Operation
|
||||
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
}
|
||||
|
||||
@Test def testCopyOnWriteStorage() {
|
||||
// Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
// Snapshot query
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF1.count())
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// Upsert Operation
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size())
|
||||
|
||||
// Snapshot Query
|
||||
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF2.count()) // still 100, since we only updated
|
||||
|
||||
// Read Incremental Query
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, hoodieIncViewDF1.count()) // 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// Upsert an empty dataFrame
|
||||
val emptyRecords = recordsToStrings(dataGen.generateUpdates("002", 0)).toList
|
||||
val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1))
|
||||
emptyDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2016/*/*/*")
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count())
|
||||
|
||||
val timeTravelDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, timeTravelDF.count()) // 100 initial inserts must be pulled
|
||||
}
|
||||
|
||||
@Test def testDropInsertDup(): Unit = {
|
||||
val insert1Cnt = 10
|
||||
val insert2DupKeyCnt = 9
|
||||
val insert2NewKeyCnt = 2
|
||||
|
||||
val totalUniqueKeyToGenerate = insert1Cnt + insert2NewKeyCnt
|
||||
val allRecords = dataGen.generateInserts("001", totalUniqueKeyToGenerate)
|
||||
val inserts1 = allRecords.subList(0, insert1Cnt)
|
||||
val inserts2New = dataGen.generateSameKeyInserts("002", allRecords.subList(insert1Cnt, insert1Cnt + insert2NewKeyCnt))
|
||||
val inserts2Dup = dataGen.generateSameKeyInserts("002", inserts1.subList(0, insert2DupKeyCnt))
|
||||
|
||||
val records1 = recordsToStrings(inserts1).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(insert1Cnt, hoodieROViewDF1.count())
|
||||
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val records2 = recordsToStrings(inserts2Dup ++ inserts2New).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate)
|
||||
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
}
|
||||
}
|
||||
@@ -1,337 +0,0 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.TableNotFoundException
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
import org.junit.jupiter.api.{BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.{Await, Future}
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource
|
||||
*/
|
||||
class TestDataSource {
|
||||
private val log = LogManager.getLogger(getClass)
|
||||
|
||||
var spark: SparkSession = null
|
||||
var dataGen: HoodieTestDataGenerator = null
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
var basePath: String = null
|
||||
var fs: FileSystem = null
|
||||
|
||||
@BeforeEach def initialize(@TempDir tempDir: java.nio.file.Path) {
|
||||
spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
dataGen = new HoodieTestDataGenerator()
|
||||
basePath = tempDir.toAbsolutePath.toString
|
||||
fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
}
|
||||
|
||||
@Test def testShortNameStorage() {
|
||||
// Insert Operation
|
||||
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
}
|
||||
|
||||
@Test def testCopyOnWriteStorage() {
|
||||
// Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*");
|
||||
assertEquals(100, hoodieROViewDF1.count())
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// Upsert Operation
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size())
|
||||
|
||||
// Read RO View
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*");
|
||||
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
|
||||
|
||||
// Read Incremental View
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0);
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(basePath);
|
||||
assertEquals(100, hoodieIncViewDF1.count()) // 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// Upsert an empty dataFrame
|
||||
val emptyRecords = recordsToStrings(dataGen.generateUpdates("002", 0)).toList
|
||||
val emptyDF: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1))
|
||||
emptyDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2016/*/*/*")
|
||||
.load(basePath);
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count())
|
||||
}
|
||||
|
||||
@Test def testMergeOnReadStorage() {
|
||||
// Bulk Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hoodieROViewDF1.count()) // still 100, since we only updated
|
||||
}
|
||||
|
||||
@Test def testDropInsertDup(): Unit = {
|
||||
val insert1Cnt = 10
|
||||
val insert2DupKeyCnt = 9
|
||||
val insert2NewKeyCnt = 2
|
||||
|
||||
val totalUniqueKeyToGenerate = insert1Cnt + insert2NewKeyCnt
|
||||
val allRecords = dataGen.generateInserts("001", totalUniqueKeyToGenerate)
|
||||
val inserts1 = allRecords.subList(0, insert1Cnt)
|
||||
val inserts2New = dataGen.generateSameKeyInserts("002", allRecords.subList(insert1Cnt, insert1Cnt + insert2NewKeyCnt))
|
||||
val inserts2Dup = dataGen.generateSameKeyInserts("002", inserts1.subList(0, insert2DupKeyCnt))
|
||||
|
||||
val records1 = recordsToStrings(inserts1).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(insert1Cnt, hoodieROViewDF1.count())
|
||||
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val records2 = recordsToStrings(inserts2Dup ++ inserts2New).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate)
|
||||
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStructuredStreaming(): Unit = {
|
||||
fs.delete(new Path(basePath), true)
|
||||
val sourcePath = basePath + "/source"
|
||||
val destPath = basePath + "/dest"
|
||||
fs.mkdirs(new Path(sourcePath))
|
||||
|
||||
// First chunk of data
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
|
||||
// Second chunk of data
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// define the source of streaming
|
||||
val streamingInput =
|
||||
spark.readStream
|
||||
.schema(inputDF1.schema)
|
||||
.json(sourcePath)
|
||||
|
||||
val f1 = Future {
|
||||
println("streaming starting")
|
||||
//'writeStream' can be called only on streaming Dataset/DataFrame
|
||||
streamingInput
|
||||
.writeStream
|
||||
.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.trigger(new ProcessingTime(100))
|
||||
.option("checkpointLocation", basePath + "/checkpoint")
|
||||
.outputMode(OutputMode.Append)
|
||||
.start(destPath)
|
||||
.awaitTermination(10000)
|
||||
println("streaming ends")
|
||||
}
|
||||
|
||||
val f2 = Future {
|
||||
inputDF1.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
val currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5);
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000"))
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assert(hoodieROViewDF1.count() == 100)
|
||||
|
||||
inputDF2.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5);
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size())
|
||||
// Read RO View
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
|
||||
|
||||
|
||||
// Read Incremental View
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(destPath)
|
||||
assertEquals(100, hoodieIncViewDF1.count())
|
||||
// 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(destPath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
}
|
||||
Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf)
|
||||
}
|
||||
|
||||
@throws[InterruptedException]
|
||||
private def waitTillAtleastNCommits(fs: FileSystem, tablePath: String,
|
||||
numCommits: Int, timeoutSecs: Int, sleepSecsAfterEachRun: Int): Int = {
|
||||
val beginTime = System.currentTimeMillis
|
||||
var currTime = beginTime
|
||||
val timeoutMsecs = timeoutSecs * 1000
|
||||
var numInstants = 0
|
||||
var success: Boolean = false
|
||||
while ({!success && (currTime - beginTime) < timeoutMsecs}) try {
|
||||
val timeline = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, tablePath)
|
||||
log.info("Timeline :" + timeline.getInstants.toArray)
|
||||
if (timeline.countInstants >= numCommits) {
|
||||
numInstants = timeline.countInstants
|
||||
success = true
|
||||
}
|
||||
val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath, true)
|
||||
} catch {
|
||||
case te: TableNotFoundException =>
|
||||
log.info("Got table not found exception. Retrying")
|
||||
} finally {
|
||||
Thread.sleep(sleepSecsAfterEachRun * 1000)
|
||||
currTime = System.currentTimeMillis
|
||||
}
|
||||
if (!success) {
|
||||
throw new IllegalStateException("Timed-out waiting for " + numCommits + " commits to appear in " + tablePath)
|
||||
}
|
||||
numInstants
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,391 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Tests on Spark DataSource for MOR table.
|
||||
*/
|
||||
class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
var spark: SparkSession = null
|
||||
private val log = LogManager.getLogger(classOf[TestMORDataSource])
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test def testMergeOnReadStorage() {
|
||||
|
||||
val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
// Bulk Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
|
||||
// Read RO View
|
||||
val hudiRODF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF1.count()) // still 100, since we only updated
|
||||
val insertCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val insertCommitTimes = hudiRODF1.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
assertEquals(List(insertCommitTime), insertCommitTimes)
|
||||
|
||||
// Upsert operation
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Read Snapshot query
|
||||
val updateCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
assertEquals(List(updateCommitTime), updateCommitTimes)
|
||||
}
|
||||
|
||||
@Test def testCount() {
|
||||
// First Operation:
|
||||
// Producing parquet files to three default partitions.
|
||||
// SNAPSHOT view on MOR table with parquet files only.
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
// Second Operation:
|
||||
// Upsert the update to the default partitions with duplicate records. Produced a log file for each parquet.
|
||||
// SNAPSHOT view should read the log files only with the latest commit time.
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count()) // still 100, since we only updated
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
val commit2Time = hudiSnapshotDF2.select("_hoodie_commit_time").head().get(0).toString
|
||||
assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1)
|
||||
assertTrue(commit2Time > commit1Time)
|
||||
assertEquals(100, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// Unmerge
|
||||
val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotSkipMergeDF2.count())
|
||||
assertEquals(100, hudiSnapshotSkipMergeDF2.select("_hoodie_record_key").distinct().count())
|
||||
assertEquals(200, hudiSnapshotSkipMergeDF2.join(hudiSnapshotDF2, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// Test Read Optimized Query on MOR table
|
||||
val hudiRODF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF2.count())
|
||||
|
||||
// Third Operation:
|
||||
// Upsert another update to the default partitions with 50 duplicate records. Produced the second log file for each parquet.
|
||||
// SNAPSHOT view should read the latest log files.
|
||||
val records3 = recordsToStrings(dataGen.generateUniqueUpdates("003", 50)).toList
|
||||
val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// still 100, because we only updated the existing records
|
||||
assertEquals(100, hudiSnapshotDF3.count())
|
||||
|
||||
// 50 from commit2, 50 from commit3
|
||||
assertEquals(hudiSnapshotDF3.select("_hoodie_commit_time").distinct().count(), 2)
|
||||
assertEquals(50, hudiSnapshotDF3.filter(col("_hoodie_commit_time") > commit2Time).count())
|
||||
assertEquals(50,
|
||||
hudiSnapshotDF3.join(hudiSnapshotDF2, Seq("_hoodie_record_key", "_hoodie_commit_time"), "inner").count())
|
||||
|
||||
// Fourth Operation:
|
||||
// Insert records to a new partition. Produced a new parquet file.
|
||||
// SNAPSHOT view should read the latest log files from the default partition and parquet from the new partition.
|
||||
val partitionPaths = new Array[String](1)
|
||||
partitionPaths.update(0, "2020/01/10")
|
||||
val newDataGen = new HoodieTestDataGenerator(partitionPaths)
|
||||
val records4 = recordsToStrings(newDataGen.generateInserts("004", 100)).toList
|
||||
val inputDF4: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records4, 2))
|
||||
inputDF4.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF4 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// 200, because we insert 100 records to a new partition
|
||||
assertEquals(200, hudiSnapshotDF4.count())
|
||||
assertEquals(100,
|
||||
hudiSnapshotDF1.join(hudiSnapshotDF4, Seq("_hoodie_record_key"), "inner").count())
|
||||
|
||||
// Fifth Operation:
|
||||
// Upsert records to the new partition. Produced a newer version of parquet file.
|
||||
// SNAPSHOT view should read the latest log files from the default partition
|
||||
// and the latest parquet from the new partition.
|
||||
val records5 = recordsToStrings(newDataGen.generateUpdates("005", 100)).toList
|
||||
val inputDF5: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records5, 2))
|
||||
inputDF5.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF5 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotDF5.count())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPayloadDelete() {
|
||||
// First Operation:
|
||||
// Producing parquet files to three default partitions.
|
||||
// SNAPSHOT view on MOR table with parquet files only.
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
// Second Operation:
|
||||
// Upsert 50 delete records
|
||||
// Snopshot view should only read 50 records
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueDeleteRecords("002", 50)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(50, hudiSnapshotDF2.count()) // 50 records were deleted
|
||||
assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1)
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
val commit2Time = hudiSnapshotDF2.select("_hoodie_commit_time").head().get(0).toString
|
||||
assertTrue(commit1Time.equals(commit2Time))
|
||||
assertEquals(50, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// unmerge query, skip the delete records
|
||||
val hudiSnapshotDF2Unmerge = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2Unmerge.count())
|
||||
|
||||
// Third Operation:
|
||||
// Upsert 50 delete records to delete the reset
|
||||
// Snopshot view should read 0 record
|
||||
val records3 = recordsToStrings(dataGen.generateUniqueDeleteRecords("003", 50)).toList
|
||||
val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(0, hudiSnapshotDF3.count()) // 100 records were deleted, 0 record to load
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPrunedFiltered() {
|
||||
// First Operation:
|
||||
// Producing parquet files to three default partitions.
|
||||
// SNAPSHOT view on MOR table with parquet files only.
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
hudiSnapshotDF1
|
||||
.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
|
||||
// Second Operation:
|
||||
// Upsert 50 update records
|
||||
// Snopshot view should read 100 records
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 50))
|
||||
.toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
|
||||
// filter first commit and only read log records
|
||||
assertEquals(50, hudiSnapshotDF2.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history")
|
||||
.filter(col("_hoodie_commit_time") > commit1Time).count())
|
||||
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
hudiSnapshotDF2
|
||||
.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
|
||||
// Correctly loading type
|
||||
val sampleRow = hudiSnapshotDF2
|
||||
.select("begin_lat", "current_date", "fare.currency", "tip_history", "nation")
|
||||
.orderBy(desc("_hoodie_commit_time"))
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getLong(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getString(2), sampleRow.get(2))
|
||||
assertEquals(sampleRow.getSeq(3), sampleRow.get(3))
|
||||
assertEquals(sampleRow.getStruct(4), sampleRow.get(4))
|
||||
|
||||
// make sure show() work
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testVectorizedReader() {
|
||||
spark.conf.set("spark.sql.parquet.enableVectorizedReader", true)
|
||||
assertTrue(spark.conf.get("spark.sql.parquet.enableVectorizedReader").toBoolean)
|
||||
// Vectorized Reader will only be triggered with AtomicType schema,
|
||||
// which is not null, UDTs, arrays, structs, and maps.
|
||||
val schema = HoodieTestDataGenerator.SHORT_TRIP_SCHEMA
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 100, schema)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdatesAsPerSchema("002", 50, schema))
|
||||
.toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count())
|
||||
|
||||
// loading correct type
|
||||
val sampleRow = hudiSnapshotDF2
|
||||
.select("fare", "driver", "_hoodie_is_deleted")
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getString(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getBoolean(2), sampleRow.get(2))
|
||||
|
||||
// test show()
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,180 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.TableNotFoundException
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.{Await, Future}
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for structured streaming sink
|
||||
*/
|
||||
class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
private val log = LogManager.getLogger(getClass)
|
||||
var spark: SparkSession = null
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStructuredStreaming(): Unit = {
|
||||
fs.delete(new Path(basePath), true)
|
||||
val sourcePath = basePath + "/source"
|
||||
val destPath = basePath + "/dest"
|
||||
fs.mkdirs(new Path(sourcePath))
|
||||
|
||||
// First chunk of data
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
|
||||
// Second chunk of data
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// define the source of streaming
|
||||
val streamingInput =
|
||||
spark.readStream
|
||||
.schema(inputDF1.schema)
|
||||
.json(sourcePath)
|
||||
|
||||
val f1 = Future {
|
||||
println("streaming starting")
|
||||
//'writeStream' can be called only on streaming Dataset/DataFrame
|
||||
streamingInput
|
||||
.writeStream
|
||||
.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.trigger(new ProcessingTime(100))
|
||||
.option("checkpointLocation", basePath + "/checkpoint")
|
||||
.outputMode(OutputMode.Append)
|
||||
.start(destPath)
|
||||
.awaitTermination(10000)
|
||||
println("streaming ends")
|
||||
}
|
||||
|
||||
val f2 = Future {
|
||||
inputDF1.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
val currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000"))
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assert(hoodieROViewDF1.count() == 100)
|
||||
|
||||
inputDF2.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5)
|
||||
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size())
|
||||
// Read RO View
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
|
||||
|
||||
|
||||
// Read Incremental View
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(destPath)
|
||||
assertEquals(100, hoodieIncViewDF1.count())
|
||||
// 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(destPath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
}
|
||||
Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf)
|
||||
}
|
||||
|
||||
@throws[InterruptedException]
|
||||
private def waitTillAtleastNCommits(fs: FileSystem, tablePath: String,
|
||||
numCommits: Int, timeoutSecs: Int, sleepSecsAfterEachRun: Int) = {
|
||||
val beginTime = System.currentTimeMillis
|
||||
var currTime = beginTime
|
||||
val timeoutMsecs = timeoutSecs * 1000
|
||||
var numInstants = 0
|
||||
var success = false
|
||||
while ({!success && (currTime - beginTime) < timeoutMsecs}) try {
|
||||
val timeline = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, tablePath)
|
||||
log.info("Timeline :" + timeline.getInstants.toArray)
|
||||
if (timeline.countInstants >= numCommits) {
|
||||
numInstants = timeline.countInstants
|
||||
success = true
|
||||
}
|
||||
val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath, true)
|
||||
} catch {
|
||||
case te: TableNotFoundException =>
|
||||
log.info("Got table not found exception. Retrying")
|
||||
} finally {
|
||||
Thread.sleep(sleepSecsAfterEachRun * 1000)
|
||||
currTime = System.currentTimeMillis
|
||||
}
|
||||
if (!success) throw new IllegalStateException("Timed-out waiting for " + numCommits + " commits to appear in " + tablePath)
|
||||
numInstants
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user