[HUDI-1371] [HUDI-1893] Support metadata based listing for Spark DataSource and Spark SQL (#2893)
This commit is contained in:
@@ -72,7 +72,7 @@ class DefaultSource extends RelationProvider
|
||||
optParams: Map[String, String],
|
||||
schema: StructType): BaseRelation = {
|
||||
// Add default options for unspecified read options keys.
|
||||
val parameters = DataSourceOptionsHelper.translateConfigurations(optParams)
|
||||
val parameters = DataSourceOptionsHelper.parametersWithReadDefaults(optParams)
|
||||
|
||||
val path = parameters.get("path")
|
||||
val readPathsStr = parameters.get(DataSourceReadOptions.READ_PATHS_OPT_KEY.key)
|
||||
@@ -106,12 +106,7 @@ class DefaultSource extends RelationProvider
|
||||
val metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(tablePath).build()
|
||||
val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent
|
||||
val tableType = metaClient.getTableType
|
||||
|
||||
// First check if the ConfigUtils.IS_QUERY_AS_RO_TABLE has set by HiveSyncTool,
|
||||
// or else use query type from QUERY_TYPE_OPT_KEY.
|
||||
val queryType = parameters.get(ConfigUtils.IS_QUERY_AS_RO_TABLE)
|
||||
.map(is => if (is.toBoolean) QUERY_TYPE_READ_OPTIMIZED_OPT_VAL else QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.getOrElse(parameters.getOrElse(QUERY_TYPE_OPT_KEY.key, QUERY_TYPE_OPT_KEY.defaultValue()))
|
||||
val queryType = parameters(QUERY_TYPE_OPT_KEY.key)
|
||||
|
||||
log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType, queryType is: $queryType")
|
||||
|
||||
|
||||
@@ -21,14 +21,14 @@ import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE_OPT_KEY, QUERY_TYPE_SNAPSHOT_OPT_VAL}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.{HoodieMetadataConfig, SerializableConfiguration}
|
||||
import org.apache.hudi.common.engine.HoodieLocalEngineContext
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieBaseFile
|
||||
import org.apache.hudi.common.model.{FileSlice, HoodieLogFile}
|
||||
import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
|
||||
@@ -76,6 +76,11 @@ case class HoodieFileIndex(
|
||||
private val basePath = metaClient.getBasePath
|
||||
|
||||
@transient private val queryPath = new Path(options.getOrElse("path", "'path' option required"))
|
||||
|
||||
private val queryType = options(QUERY_TYPE_OPT_KEY.key())
|
||||
|
||||
private val tableType = metaClient.getTableType
|
||||
|
||||
/**
|
||||
* Get the schema of the table.
|
||||
*/
|
||||
@@ -106,16 +111,35 @@ case class HoodieFileIndex(
|
||||
}
|
||||
}
|
||||
|
||||
private lazy val metadataConfig = {
|
||||
private lazy val engineContext = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext))
|
||||
|
||||
private lazy val configProperties = {
|
||||
val sqlConf: SQLConf = spark.sessionState.conf
|
||||
val properties = new Properties()
|
||||
|
||||
// To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users
|
||||
// would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing.
|
||||
properties.put(HoodieMetadataConfig.METADATA_ENABLE_PROP,
|
||||
sqlConf.getConfString(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(),
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString))
|
||||
properties.put(HoodieMetadataConfig.METADATA_VALIDATE_PROP,
|
||||
sqlConf.getConfString(HoodieMetadataConfig.METADATA_VALIDATE_PROP.key(),
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue().toString))
|
||||
properties.putAll(options.asJava)
|
||||
HoodieMetadataConfig.newBuilder.fromProperties(properties).build()
|
||||
properties
|
||||
}
|
||||
|
||||
private lazy val fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
|
||||
.fromProperties(configProperties)
|
||||
.build()
|
||||
|
||||
private lazy val metadataConfig = HoodieMetadataConfig.newBuilder
|
||||
.fromProperties(configProperties)
|
||||
.build()
|
||||
|
||||
@transient @volatile private var fileSystemView: HoodieTableFileSystemView = _
|
||||
@transient @volatile private var cachedAllInputFiles: Array[HoodieBaseFile] = _
|
||||
@transient @volatile private var cachedAllInputFileSlices: Map[PartitionRowPath, Seq[FileSlice]] = _
|
||||
@transient @volatile private var cachedFileSize: Long = 0L
|
||||
@transient @volatile private var cachedAllPartitionPaths: Seq[PartitionRowPath] = _
|
||||
|
||||
@volatile private var queryAsNonePartitionedTable: Boolean = _
|
||||
|
||||
@@ -123,24 +147,58 @@ case class HoodieFileIndex(
|
||||
|
||||
override def rootPaths: Seq[Path] = queryPath :: Nil
|
||||
|
||||
/**
|
||||
* Invoked by Spark to fetch list of latest base files per partition.
|
||||
*
|
||||
* @param partitionFilters partition column filters
|
||||
* @param dataFilters data columns filters
|
||||
* @return list of PartitionDirectory containing partition to base files mapping
|
||||
*/
|
||||
override def listFiles(partitionFilters: Seq[Expression],
|
||||
dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
|
||||
if (queryAsNonePartitionedTable) { // Read as Non-Partitioned table.
|
||||
Seq(PartitionDirectory(InternalRow.empty, allFiles))
|
||||
} else {
|
||||
// Prune the partition path by the partition filters
|
||||
val prunedPartitions = prunePartition(cachedAllPartitionPaths, partitionFilters)
|
||||
val prunedPartitions = prunePartition(cachedAllInputFileSlices.keys.toSeq, partitionFilters)
|
||||
prunedPartitions.map { partition =>
|
||||
val fileStatues = fileSystemView.getLatestBaseFiles(partition.partitionPath).iterator()
|
||||
.asScala.toSeq
|
||||
.map(_.getFileStatus)
|
||||
PartitionDirectory(partition.values, fileStatues)
|
||||
val baseFileStatuses = cachedAllInputFileSlices(partition).map(fileSlice => {
|
||||
if (fileSlice.getBaseFile.isPresent) {
|
||||
fileSlice.getBaseFile.get().getFileStatus
|
||||
} else {
|
||||
null
|
||||
}
|
||||
}).filterNot(_ == null)
|
||||
|
||||
PartitionDirectory(partition.values, baseFileStatuses)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch list of latest base files and log files per partition.
|
||||
*
|
||||
* @param partitionFilters partition column filters
|
||||
* @param dataFilters data column filters
|
||||
* @return mapping from string partition paths to its base/log files
|
||||
*/
|
||||
def listFileSlices(partitionFilters: Seq[Expression],
|
||||
dataFilters: Seq[Expression]): Map[String, Seq[FileSlice]] = {
|
||||
if (queryAsNonePartitionedTable) {
|
||||
// Read as Non-Partitioned table.
|
||||
cachedAllInputFileSlices.map(entry => (entry._1.partitionPath, entry._2))
|
||||
} else {
|
||||
// Prune the partition path by the partition filters
|
||||
val prunedPartitions = prunePartition(cachedAllInputFileSlices.keys.toSeq, partitionFilters)
|
||||
prunedPartitions.map(partition => {
|
||||
(partition.partitionPath, cachedAllInputFileSlices(partition))
|
||||
}).toMap
|
||||
}
|
||||
}
|
||||
|
||||
override def inputFiles: Array[String] = {
|
||||
cachedAllInputFiles.map(_.getFileStatus.getPath.toString)
|
||||
val fileStatusList = allFiles
|
||||
fileStatusList.map(_.getPath.toString).toArray
|
||||
}
|
||||
|
||||
override def refresh(): Unit = {
|
||||
@@ -157,13 +215,36 @@ case class HoodieFileIndex(
|
||||
metaClient.reloadActiveTimeline()
|
||||
val activeInstants = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
|
||||
fileSystemView = new HoodieTableFileSystemView(metaClient, activeInstants, allFiles)
|
||||
cachedAllInputFiles = fileSystemView.getLatestBaseFiles.iterator().asScala.toArray
|
||||
cachedAllPartitionPaths = partitionFiles.keys.toSeq
|
||||
cachedFileSize = cachedAllInputFiles.map(_.getFileLen).sum
|
||||
|
||||
(tableType, queryType) match {
|
||||
case (MERGE_ON_READ, QUERY_TYPE_SNAPSHOT_OPT_VAL) =>
|
||||
// Fetch and store latest base and log files, and their sizes
|
||||
cachedAllInputFileSlices = partitionFiles.map(p => {
|
||||
val latestSlices = if (activeInstants.lastInstant().isPresent) {
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(p._1.partitionPath,
|
||||
activeInstants.lastInstant().get().getTimestamp).iterator().asScala.toSeq
|
||||
} else {
|
||||
Seq()
|
||||
}
|
||||
(p._1, latestSlices)
|
||||
})
|
||||
cachedFileSize = cachedAllInputFileSlices.values.flatten.map(fileSlice => {
|
||||
if (fileSlice.getBaseFile.isPresent) {
|
||||
fileSlice.getBaseFile.get().getFileLen + fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).sum
|
||||
} else {
|
||||
fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).sum
|
||||
}
|
||||
}).sum
|
||||
case (_, _) =>
|
||||
// Fetch and store latest base files and its sizes
|
||||
cachedAllInputFileSlices = partitionFiles.map(p => {
|
||||
(p._1, fileSystemView.getLatestFileSlices(p._1.partitionPath).iterator().asScala.toSeq)
|
||||
})
|
||||
cachedFileSize = cachedAllInputFileSlices.values.flatten.map(_.getBaseFile.get().getFileLen).sum
|
||||
}
|
||||
|
||||
// If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
|
||||
queryAsNonePartitionedTable = cachedAllPartitionPaths
|
||||
.exists(p => p.values == InternalRow.empty)
|
||||
queryAsNonePartitionedTable = partitionFiles.keys.exists(p => p.values == InternalRow.empty)
|
||||
val flushSpend = System.currentTimeMillis() - startTime
|
||||
logInfo(s"Refresh for table ${metaClient.getTableConfig.getTableName}," +
|
||||
s" spend: $flushSpend ms")
|
||||
@@ -192,7 +273,20 @@ case class HoodieFileIndex(
|
||||
StructType(schema.fields.filterNot(f => partitionColumns.contains(f.name)))
|
||||
}
|
||||
|
||||
def allFiles: Seq[FileStatus] = cachedAllInputFiles.map(_.getFileStatus)
|
||||
/**
|
||||
* Returns the FileStatus for all the base files (excluding log files). This should be used only for
|
||||
* cases where Spark directly fetches the list of files via HoodieFileIndex or for read optimized query logic
|
||||
* implemented internally within Hudi like HoodieBootstrapRelation. This helps avoid the use of path filter
|
||||
* to filter out log files within Spark.
|
||||
*
|
||||
* @return List of FileStatus for base files
|
||||
*/
|
||||
def allFiles: Seq[FileStatus] = {
|
||||
cachedAllInputFileSlices.values.flatten
|
||||
.filter(_.getBaseFile.isPresent)
|
||||
.map(_.getBaseFile.get().getFileStatus)
|
||||
.toSeq
|
||||
}
|
||||
|
||||
/**
|
||||
* Prune the partition by the filter.This implementation is fork from
|
||||
@@ -229,12 +323,12 @@ case class HoodieFileIndex(
|
||||
}
|
||||
|
||||
def getAllQueryPartitionPaths: Seq[PartitionRowPath] = {
|
||||
val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext))
|
||||
val queryPartitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), queryPath)
|
||||
// Load all the partition path from the basePath, and filter by the query partition path.
|
||||
// TODO load files from the queryPartitionPath directly.
|
||||
val partitionPaths = FSUtils.getAllPartitionPaths(sparkEngine, metadataConfig, basePath).asScala
|
||||
val partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath).asScala
|
||||
.filter(_.startsWith(queryPartitionPath))
|
||||
|
||||
val partitionSchema = _partitionSchemaFromProperties
|
||||
val timeZoneId = CaseInsensitiveMap(options)
|
||||
.get(DateTimeUtils.TIMEZONE_OPTION)
|
||||
@@ -307,13 +401,6 @@ case class HoodieFileIndex(
|
||||
* Load all partition paths and it's files under the query table path.
|
||||
*/
|
||||
private def loadPartitionPathFiles(): Map[PartitionRowPath, Array[FileStatus]] = {
|
||||
val properties = new Properties()
|
||||
properties.putAll(options.asJava)
|
||||
val writeConfig = HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath).withProperties(properties).build()
|
||||
|
||||
val maxListParallelism = writeConfig.getFileListingParallelism
|
||||
val serializableConf = new SerializableConfiguration(spark.sessionState.newHadoopConf())
|
||||
val partitionRowPaths = getAllQueryPartitionPaths
|
||||
// List files in all of the partition path.
|
||||
val pathToFetch = mutable.ArrayBuffer[PartitionRowPath]()
|
||||
@@ -327,28 +414,25 @@ case class HoodieFileIndex(
|
||||
case None => pathToFetch.append(partitionRowPath)
|
||||
}
|
||||
}
|
||||
// Fetch the rest from the file system.
|
||||
val fetchedPartition2Files =
|
||||
|
||||
val fetchedPartitionToFiles =
|
||||
if (pathToFetch.nonEmpty) {
|
||||
spark.sparkContext.parallelize(pathToFetch, Math.min(pathToFetch.size, maxListParallelism))
|
||||
.map { partitionRowPath =>
|
||||
// Here we use a LocalEngineContext to get the files in the partition.
|
||||
// We can do this because the TableMetadata.getAllFilesInPartition only rely on the
|
||||
// hadoopConf of the EngineContext.
|
||||
val engineContext = new HoodieLocalEngineContext(serializableConf.get())
|
||||
val filesInPartition = FSUtils.getFilesInPartition(engineContext, metadataConfig,
|
||||
basePath, partitionRowPath.fullPartitionPath(basePath))
|
||||
(partitionRowPath, filesInPartition)
|
||||
}.collect().map(f => f._1 -> f._2).toMap
|
||||
val fullPartitionPathsToFetch = pathToFetch.map(p => (p, p.fullPartitionPath(basePath).toString)).toMap
|
||||
val partitionToFilesMap = FSUtils.getFilesInPartitions(engineContext, metadataConfig, basePath,
|
||||
fullPartitionPathsToFetch.values.toArray, fileSystemStorageConfig.getSpillableDir)
|
||||
fullPartitionPathsToFetch.map(p => {
|
||||
(p._1, partitionToFilesMap.get(p._2))
|
||||
})
|
||||
} else {
|
||||
Map.empty[PartitionRowPath, Array[FileStatus]]
|
||||
}
|
||||
|
||||
// Update the fileStatusCache
|
||||
fetchedPartition2Files.foreach {
|
||||
fetchedPartitionToFiles.foreach {
|
||||
case (partitionRowPath, filesInPartition) =>
|
||||
fileStatusCache.putLeafFiles(partitionRowPath.fullPartitionPath(basePath), filesInPartition)
|
||||
}
|
||||
cachePartitionToFiles.toMap ++ fetchedPartition2Files
|
||||
cachePartitionToFiles.toMap ++ fetchedPartitionToFiles
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.model.HoodieLogFile
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
|
||||
@@ -142,15 +143,45 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
|
||||
}
|
||||
|
||||
def buildFileIndex(filters: Array[Filter]): List[HoodieMergeOnReadFileSplit] = {
|
||||
// Get all partition paths
|
||||
val partitionPaths = if (globPaths.isDefined) {
|
||||
if (globPaths.isDefined) {
|
||||
// Load files from the global paths if it has defined to be compatible with the original mode
|
||||
val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(sqlContext.sparkSession, globPaths.get)
|
||||
val fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline.getCommitsTimeline
|
||||
.filterCompletedInstants, inMemoryFileIndex.allFiles().toArray)
|
||||
fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent)
|
||||
} else { // Load partition path by the HoodieFileIndex.
|
||||
val partitionPaths = fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent)
|
||||
|
||||
|
||||
if (partitionPaths.isEmpty) { // If this an empty table, return an empty split list.
|
||||
List.empty[HoodieMergeOnReadFileSplit]
|
||||
} else {
|
||||
val lastInstant = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants.lastInstant()
|
||||
if (!lastInstant.isPresent) { // Return empty list if the table has no commit
|
||||
List.empty
|
||||
} else {
|
||||
val latestCommit = lastInstant.get().getTimestamp
|
||||
val baseAndLogsList = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, partitionPaths.asJava).asScala
|
||||
val fileSplits = baseAndLogsList.map(kv => {
|
||||
val baseFile = kv.getLeft
|
||||
val logPaths = if (kv.getRight.isEmpty) Option.empty else Option(kv.getRight.asScala.toList)
|
||||
|
||||
val baseDataPath = if (baseFile.isPresent) {
|
||||
Some(PartitionedFile(
|
||||
InternalRow.empty,
|
||||
MergeOnReadSnapshotRelation.getFilePath(baseFile.get.getFileStatus.getPath),
|
||||
0, baseFile.get.getFileLen)
|
||||
)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
HoodieMergeOnReadFileSplit(baseDataPath, logPaths, latestCommit,
|
||||
metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType)
|
||||
}).toList
|
||||
fileSplits
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Load files by the HoodieFileIndex.
|
||||
val hoodieFileIndex = HoodieFileIndex(sqlContext.sparkSession, metaClient,
|
||||
Some(tableStructSchema), optParams, FileStatusCache.getOrCreate(sqlContext.sparkSession))
|
||||
|
||||
@@ -160,36 +191,34 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
|
||||
val partitionFilterExpression =
|
||||
HoodieSparkUtils.convertToCatalystExpressions(partitionFilters, tableStructSchema)
|
||||
|
||||
val allPartitionPaths = hoodieFileIndex.getAllQueryPartitionPaths
|
||||
// If convert success to catalyst expression, use the partition prune
|
||||
hoodieFileIndex.prunePartition(allPartitionPaths, partitionFilterExpression.map(Seq(_)).getOrElse(Seq.empty))
|
||||
.map(_.fullPartitionPath(metaClient.getBasePath))
|
||||
}
|
||||
|
||||
if (partitionPaths.isEmpty) { // If this an empty table, return an empty split list.
|
||||
List.empty[HoodieMergeOnReadFileSplit]
|
||||
} else {
|
||||
val lastInstant = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants.lastInstant()
|
||||
if (!lastInstant.isPresent) { // Return empty list if the table has no commit
|
||||
List.empty
|
||||
val fileSlices = if (partitionFilterExpression.isDefined) {
|
||||
hoodieFileIndex.listFileSlices(Seq(partitionFilterExpression.get), Seq.empty)
|
||||
} else {
|
||||
val latestCommit = lastInstant.get().getTimestamp
|
||||
val baseAndLogsList = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, partitionPaths.asJava).asScala
|
||||
val fileSplits = baseAndLogsList.map(kv => {
|
||||
val baseFile = kv.getLeft
|
||||
val logPaths = if (kv.getRight.isEmpty) Option.empty else Option(kv.getRight.asScala.toList)
|
||||
hoodieFileIndex.listFileSlices(Seq.empty, Seq.empty)
|
||||
}
|
||||
|
||||
val baseDataPath = if (baseFile.isPresent) {
|
||||
Some(PartitionedFile(
|
||||
InternalRow.empty,
|
||||
MergeOnReadSnapshotRelation.getFilePath(baseFile.get.getFileStatus.getPath),
|
||||
0, baseFile.get.getFileLen)
|
||||
)
|
||||
if (fileSlices.isEmpty) {
|
||||
// If this an empty table, return an empty split list.
|
||||
List.empty[HoodieMergeOnReadFileSplit]
|
||||
} else {
|
||||
val fileSplits = fileSlices.values.flatten.map(fileSlice => {
|
||||
val latestCommit = metaClient.getActiveTimeline.getCommitsTimeline
|
||||
.filterCompletedInstants.lastInstant().get().getTimestamp
|
||||
|
||||
val partitionedFile = if (fileSlice.getBaseFile.isPresent) {
|
||||
val baseFile = fileSlice.getBaseFile.get()
|
||||
val baseFilePath = MergeOnReadSnapshotRelation.getFilePath(baseFile.getFileStatus.getPath)
|
||||
Option(PartitionedFile(InternalRow.empty, baseFilePath, 0, baseFile.getFileLen))
|
||||
} else {
|
||||
None
|
||||
Option.empty
|
||||
}
|
||||
HoodieMergeOnReadFileSplit(baseDataPath, logPaths, latestCommit,
|
||||
metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType)
|
||||
|
||||
val logPaths = fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala
|
||||
.map(logFile => MergeOnReadSnapshotRelation.getFilePath(logFile.getPath)).toList
|
||||
val logPathsOptional = if (logPaths.isEmpty) Option.empty else Option(logPaths)
|
||||
HoodieMergeOnReadFileSplit(partitionedFile, logPathsOptional, latestCommit, metaClient.getBasePath,
|
||||
maxCompactionMemoryInBytes, mergeType)
|
||||
}).toList
|
||||
fileSplits
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user