[HUDI-2711] Fallback to fulltable scan for IncrementalRelation if underlying files have been cleared or moved by cleaner (#3946)
Co-authored-by: sivabalan <n.siva.b@gmail.com>
This commit is contained in:
@@ -34,7 +34,8 @@ import org.apache.spark.sql.functions.{col, concat, lit, udf}
|
||||
import org.apache.spark.sql.types._
|
||||
import org.joda.time.DateTime
|
||||
import org.joda.time.format.DateTimeFormat
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue, fail}
|
||||
import org.junit.jupiter.api.function.Executable
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
|
||||
@@ -708,6 +709,90 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
assertEquals(numRecords - numRecordsToDelete, snapshotDF2.count())
|
||||
}
|
||||
|
||||
@Test def testFailEarlyForIncrViewQueryForNonExistingFiles(): Unit = {
|
||||
// Create 10 commits
|
||||
for (i <- 1 to 10) {
|
||||
val records = recordsToStrings(dataGen.generateInserts("%05d".format(i), 100)).toList
|
||||
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||
inputDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.cleaner.commits.retained", "3")
|
||||
.option("hoodie.keep.min.commits", "4")
|
||||
.option("hoodie.keep.max.commits", "5")
|
||||
.option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.ENABLE.key(), value = false)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
}
|
||||
|
||||
val hoodieMetaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build()
|
||||
/**
|
||||
* State of timeline after 10 commits
|
||||
* +------------------+--------------------------------------+
|
||||
* | Archived | Active Timeline |
|
||||
* +------------------+--------------+-----------------------+
|
||||
* | C0 C1 C2 C3 | C4 C5 | C6 C7 C8 C9 |
|
||||
* +------------------+--------------+-----------------------+
|
||||
* | Data cleaned | Data exists in table |
|
||||
* +---------------------------------+-----------------------+
|
||||
*/
|
||||
|
||||
val completedCommits = hoodieMetaClient.getCommitsTimeline.filterCompletedInstants() // C4 to C9
|
||||
//Anything less than 2 is a valid commit in the sense no cleanup has been done for those commit files
|
||||
var startTs = completedCommits.nthInstant(0).get().getTimestamp //C4
|
||||
var endTs = completedCommits.nthInstant(1).get().getTimestamp //C5
|
||||
|
||||
//Calling without the fallback should result in Path does not exist
|
||||
var hoodieIncViewDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key(), startTs)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME.key(), endTs)
|
||||
.load(basePath)
|
||||
|
||||
val msg = "Should fail with Path does not exist"
|
||||
assertThrows(classOf[AnalysisException], new Executable {
|
||||
override def execute(): Unit = {
|
||||
hoodieIncViewDF.count()
|
||||
}
|
||||
}, msg)
|
||||
|
||||
//Should work with fallback enabled
|
||||
hoodieIncViewDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key(), startTs)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME.key(), endTs)
|
||||
.option(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key(), "true")
|
||||
.load(basePath)
|
||||
assertEquals(100, hoodieIncViewDF.count())
|
||||
|
||||
//Test out for archived commits
|
||||
val archivedInstants = hoodieMetaClient.getArchivedTimeline.getInstants.distinct().toArray
|
||||
startTs = archivedInstants(0).asInstanceOf[HoodieInstant].getTimestamp //C0
|
||||
endTs = completedCommits.nthInstant(1).get().getTimestamp //C5
|
||||
|
||||
//Calling without the fallback should result in Path does not exist
|
||||
hoodieIncViewDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key(), startTs)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME.key(), endTs)
|
||||
.load(basePath)
|
||||
|
||||
assertThrows(classOf[AnalysisException], new Executable {
|
||||
override def execute(): Unit = {
|
||||
hoodieIncViewDF.count()
|
||||
}
|
||||
}, msg)
|
||||
|
||||
//Should work with fallback enabled
|
||||
hoodieIncViewDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key(), startTs)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME.key(), endTs)
|
||||
.option(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key(), "true")
|
||||
.load(basePath)
|
||||
assertEquals(500, hoodieIncViewDF.count())
|
||||
}
|
||||
|
||||
def copyOnWriteTableSelect(enableDropPartitionColumns: Boolean): Boolean = {
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsContainsAllPartitions("000", 3)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
|
||||
Reference in New Issue
Block a user