[HUDI-597] Enable incremental pulling from defined partitions (#1348)
This commit is contained in:
@@ -22,6 +22,7 @@ import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime}
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.junit.Assert._
|
||||
import org.junit.rules.TemporaryFolder
|
||||
import org.junit.{Before, Test}
|
||||
@@ -135,6 +136,14 @@ class TestDataSource extends AssertionsForJUnit {
|
||||
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() {
|
||||
|
||||
Reference in New Issue
Block a user