Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-4944] use mor solution to uri filenotfound problem #8809

Merged
merged 3 commits into from
May 30, 2023
Merged
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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

/**
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -200,14 +191,6 @@ case class HoodieBootstrapRelation(override val sqlContext: SQLContext,
override def updatePrunedDataSchema(prunedSchema: StructType): HoodieBootstrapRelation =
this.copy(prunedDataSchema = Some(prunedSchema))

//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
}
}


Expand All @@ -222,4 +205,10 @@ object HoodieBootstrapRelation {
checkState(combinedColumns.sorted == requiredDataColumns.sorted)
}

def getFilePath(path: Path): String = {

//see MergeOnReadSnapshotRelation.getFilePath()
path.toUri.toString
}

}