[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:
@@ -122,6 +122,10 @@ object DataSourceReadOptions {
|
||||
.withDocumentation("Enables data-skipping allowing queries to leverage indexes to reduce the search space by " +
|
||||
"skipping over files")
|
||||
|
||||
val INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.read.incr.fallback.fulltablescan.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When doing an incremental query whether we should fall back to full table scans if file does not exist.")
|
||||
/** @deprecated Use {@link QUERY_TYPE} and its methods instead */
|
||||
@Deprecated
|
||||
val QUERY_TYPE_OPT_KEY = QUERY_TYPE.key()
|
||||
|
||||
@@ -18,16 +18,17 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieReplaceCommitMetadata}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
|
||||
import java.util.stream.Collectors
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieReplaceCommitMetadata, HoodieTableType}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hadoop.fs.GlobPattern
|
||||
import org.apache.hadoop.fs.{GlobPattern, Path}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
||||
import org.apache.hudi.common.util.HoodieTimer
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
@@ -40,11 +41,11 @@ import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
* Relation, that implements the Hoodie incremental view.
|
||||
*
|
||||
* Implemented for Copy_on_write storage.
|
||||
*
|
||||
*/
|
||||
* Relation, that implements the Hoodie incremental view.
|
||||
*
|
||||
* Implemented for Copy_on_write storage.
|
||||
*
|
||||
*/
|
||||
class IncrementalRelation(val sqlContext: SQLContext,
|
||||
val optParams: Map[String, String],
|
||||
val userSchema: StructType,
|
||||
@@ -85,7 +86,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
log.info("Inferring schema..")
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val tableSchema = if (useEndInstantSchema) {
|
||||
if (commitsToReturn.isEmpty) schemaResolver.getTableAvroSchemaWithoutMetadataFields() else
|
||||
if (commitsToReturn.isEmpty) schemaResolver.getTableAvroSchemaWithoutMetadataFields() else
|
||||
schemaResolver.getTableAvroSchemaWithoutMetadataFields(commitsToReturn.last)
|
||||
} else {
|
||||
schemaResolver.getTableAvroSchemaWithoutMetadataFields()
|
||||
@@ -165,26 +166,63 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
if (filteredRegularFullPaths.isEmpty && filteredMetaBootstrapFullPaths.isEmpty) {
|
||||
sqlContext.sparkContext.emptyRDD[Row]
|
||||
} else {
|
||||
log.info("Additional Filters to be applied to incremental source are :" + filters)
|
||||
log.info("Additional Filters to be applied to incremental source are :" + filters.mkString("Array(", ", ", ")"))
|
||||
|
||||
var df: DataFrame = sqlContext.createDataFrame(sqlContext.sparkContext.emptyRDD[Row], usedSchema)
|
||||
|
||||
if (metaBootstrapFileIdToFullPath.nonEmpty) {
|
||||
df = sqlContext.sparkSession.read
|
||||
.format("hudi")
|
||||
.schema(usedSchema)
|
||||
.option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(","))
|
||||
.load()
|
||||
val fallbackToFullTableScan = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key,
|
||||
DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.defaultValue).toBoolean
|
||||
|
||||
var doFullTableScan = false
|
||||
|
||||
if (fallbackToFullTableScan) {
|
||||
val fs = new Path(basePath).getFileSystem(sqlContext.sparkContext.hadoopConfiguration);
|
||||
val timer = new HoodieTimer().startTimer();
|
||||
|
||||
val allFilesToCheck = filteredMetaBootstrapFullPaths ++ filteredRegularFullPaths
|
||||
val firstNotFoundPath = allFilesToCheck.find(path => !fs.exists(new Path(path)))
|
||||
val timeTaken = timer.endTimer()
|
||||
log.info("Checking if paths exists took " + timeTaken + "ms")
|
||||
|
||||
val optStartTs = optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key)
|
||||
val isInstantArchived = optStartTs.compareTo(commitTimeline.firstInstant().get().getTimestamp) < 0 // True if optStartTs < activeTimeline.first
|
||||
|
||||
if (isInstantArchived || firstNotFoundPath.isDefined) {
|
||||
doFullTableScan = true
|
||||
log.info("Falling back to full table scan")
|
||||
}
|
||||
}
|
||||
|
||||
if (regularFileIdToFullPath.nonEmpty) {
|
||||
df = df.union(sqlContext.read.options(sOpts)
|
||||
if (doFullTableScan) {
|
||||
val hudiDF = sqlContext.read
|
||||
.format("hudi")
|
||||
.schema(usedSchema)
|
||||
.parquet(filteredRegularFullPaths.toList: _*)
|
||||
.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.head.getTimestamp))
|
||||
.load(basePath)
|
||||
.filter(String.format("%s > '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, //Notice the > in place of >= because we are working with optParam instead of first commit > optParam
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key)))
|
||||
.filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.last.getTimestamp)))
|
||||
commitsToReturn.last.getTimestamp))
|
||||
// schema enforcement does not happen in above spark.read with hudi. hence selecting explicitly w/ right column order
|
||||
val fieldNames : Array[String] = df.schema.fields.map(field => field.name)
|
||||
df = df.union(hudiDF.select(fieldNames.head, fieldNames.tail: _*))
|
||||
} else {
|
||||
if (metaBootstrapFileIdToFullPath.nonEmpty) {
|
||||
df = sqlContext.sparkSession.read
|
||||
.format("hudi")
|
||||
.schema(usedSchema)
|
||||
.option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(","))
|
||||
.load()
|
||||
}
|
||||
|
||||
if (regularFileIdToFullPath.nonEmpty) {
|
||||
df = df.union(sqlContext.read.options(sOpts)
|
||||
.schema(usedSchema)
|
||||
.parquet(filteredRegularFullPaths.toList: _*)
|
||||
.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.head.getTimestamp))
|
||||
.filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.last.getTimestamp)))
|
||||
}
|
||||
}
|
||||
|
||||
filters.foldLeft(df)((e, f) => e.filter(f)).rdd
|
||||
|
||||
Reference in New Issue
Block a user