Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 10 additions & 2 deletions core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
Original file line number Diff line number Diff line change
Expand Up @@ -226,14 +226,22 @@ class SparkHadoopUtil extends Logging {
}

def globPath(pattern: Path): Seq[Path] = {
val fs = pattern.getFileSystem(conf)
globPath(pattern, conf)
}

def globPath(pattern: Path, hadoopConf: Configuration): Seq[Path] = {
val fs = pattern.getFileSystem(hadoopConf)
Option(fs.globStatus(pattern)).map { statuses =>
statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toSeq
}.getOrElse(Seq.empty[Path])
}

def globPathIfNecessary(pattern: Path): Seq[Path] = {
if (isGlobPath(pattern)) globPath(pattern) else Seq(pattern)
globPathIfNecessary(pattern, conf)
}

def globPathIfNecessary(pattern: Path, hadoopConf: Configuration): Seq[Path] = {
if (isGlobPath(pattern)) globPath(pattern, hadoopConf) else Seq(pattern)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ case class DataSource(
val hdfsPath = new Path(path)
val fs = hdfsPath.getFileSystem(hadoopConf)
val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
SparkHadoopUtil.get.globPathIfNecessary(qualified)
SparkHadoopUtil.get.globPathIfNecessary(qualified, hadoopConf)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you pass FileSystem into globPathIfNecessary?

}.toArray
new InMemoryFileIndex(sparkSession, globbedPaths, options, None, fileStatusCache)
}
Expand Down Expand Up @@ -364,7 +364,7 @@ case class DataSource(
val hdfsPath = new Path(path)
val fs = hdfsPath.getFileSystem(hadoopConf)
val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
val globPath = SparkHadoopUtil.get.globPathIfNecessary(qualified)
val globPath = SparkHadoopUtil.get.globPathIfNecessary(qualified, hadoopConf)

if (globPath.isEmpty) {
throw new AnalysisException(s"Path does not exist: $qualified")
Expand Down