diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala index 34db28505010..70ebd21d7e9d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala @@ -18,13 +18,11 @@ package org.apache.hudi -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieBaseRelation.{BaseFileReader, convertToAvroSchema, projectReader} -import org.apache.hudi.HoodieBootstrapRelation.validate +import org.apache.hudi.HoodieBootstrapRelation.{getFilePath, validate} import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.hadoop.CachingPath import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow @@ -34,8 +32,6 @@ import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -import java.net.URI - case class HoodieBootstrapSplit(dataFile: PartitionedFile, skeletonFile: Option[PartitionedFile] = None) extends HoodieFileSplit /** @@ -77,16 +73,11 @@ case class HoodieBootstrapRelation(override val sqlContext: SQLContext, val isPartitioned = metaClient.getTableConfig.isTablePartitioned fileSlices.map { fileSlice => val baseFile = fileSlice.getBaseFile.get() - if (baseFile.getBootstrapBaseFile.isPresent) { val partitionValues = getPartitionColumnsAsInternalRowInternal(baseFile.getBootstrapBaseFile.get.getFileStatus, bootstrapBasePath, extractPartitionValuesFromPartitionPath = isPartitioned) - val filePath = if (isPartitioned) { - encodePartitionPath(baseFile.getBootstrapBaseFile.get.getFileStatus) - } else { - baseFile.getBootstrapBaseFile.get.getPath - } - val dataFile = PartitionedFile(partitionValues, filePath, 0, baseFile.getBootstrapBaseFile.get().getFileLen) + val dataFile = PartitionedFile(partitionValues, getFilePath(baseFile.getBootstrapBaseFile.get.getFileStatus.getPath), + 0, baseFile.getBootstrapBaseFile.get().getFileLen) val skeletonFile = Option(PartitionedFile(InternalRow.empty, baseFile.getPath, 0, baseFile.getFileLen)) HoodieBootstrapSplit(dataFile, skeletonFile) @@ -210,14 +201,6 @@ case class HoodieBootstrapRelation(override val sqlContext: SQLContext, optParams)(sparkSession) } - //TODO: This should be unnecessary with spark 3.4 [SPARK-41970] - private def encodePartitionPath(file: FileStatus): String = { - val tablePathWithoutScheme = CachingPath.getPathWithoutSchemeAndAuthority(bootstrapBasePath) - val partitionPathWithoutScheme = CachingPath.getPathWithoutSchemeAndAuthority(file.getPath.getParent) - val filePathWithoutScheme = CachingPath.getPathWithoutSchemeAndAuthority(file.getPath) - val relativePath = new URI(tablePathWithoutScheme.toString).relativize(new URI(partitionPathWithoutScheme.toString)).toString - CachingPath.concatPathUnsafe(CachingPath.concatPathUnsafe(tablePathWithoutScheme, PartitionPathEncodeUtils.escapePathName(relativePath)), filePathWithoutScheme.getName).toString - } } @@ -231,4 +214,10 @@ object HoodieBootstrapRelation { checkState(combinedColumns.sorted == requiredDataColumns.sorted) } + def getFilePath(path: Path): String = { + + //see MergeOnReadSnapshotRelation.getFilePath() + path.toUri.toString + } + }