Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 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
Original file line number Diff line number Diff line change
Expand Up @@ -287,36 +287,24 @@ trait FileCatalog {
}

/**
* A file catalog that caches metadata gathered by scanning all the files present in `paths`
* recursively.
* An abstract class that represents [[FileCatalog]]s that are aware of partitioned tables.
* It provides the necessary methods to parse partition data based on a set of files.
*
* @param parameters as set of options to control discovery
* @param paths a list of paths to scan
* @param parameters as set of options to control partition discovery
* @param partitionSchema an optional partition schema that will be use to provide types for the
* discovered partitions
*/
class HDFSFileCatalog(
*/
abstract class PartitioningAwareFileCatalog(
sparkSession: SparkSession,
parameters: Map[String, String],
override val paths: Seq[Path],
partitionSchema: Option[StructType])
extends FileCatalog with Logging {

private val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters)
protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters)

var leafFiles = mutable.LinkedHashMap.empty[Path, FileStatus]
var leafDirToChildrenFiles = mutable.Map.empty[Path, Array[FileStatus]]
var cachedPartitionSpec: PartitionSpec = _
protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus]

def partitionSpec(): PartitionSpec = {
if (cachedPartitionSpec == null) {
cachedPartitionSpec = inferPartitioning(partitionSchema)
}

cachedPartitionSpec
}

refresh()
protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]]

override def listFiles(filters: Seq[Expression]): Seq[Partition] = {
if (partitionSpec().partitionColumns.isEmpty) {
Expand All @@ -331,7 +319,44 @@ class HDFSFileCatalog(
}
}

protected def prunePartitions(
override def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq

override def getStatus(path: Path): Array[FileStatus] = leafDirToChildrenFiles(path)

protected def inferPartitioning(): PartitionSpec = {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@marmbrus These methods are basically unchanged, just moved around from HDFSFileCatalog to PartitioningAwareFileCatalog.

// We use leaf dirs containing data files to discover the schema.
val leafDirs = leafDirToChildrenFiles.keys.toSeq
partitionSchema match {
case Some(userProvidedSchema) if userProvidedSchema.nonEmpty =>
val spec = PartitioningUtils.parsePartitions(
leafDirs,
PartitioningUtils.DEFAULT_PARTITION_NAME,
typeInference = false,
basePaths = basePaths)

// Without auto inference, all of value in the `row` should be null or in StringType,
// we need to cast into the data type that user specified.
def castPartitionValuesToUserSchema(row: InternalRow) = {
InternalRow((0 until row.numFields).map { i =>
Cast(
Literal.create(row.getUTF8String(i), StringType),
userProvidedSchema.fields(i).dataType).eval()
}: _*)
}

PartitionSpec(userProvidedSchema, spec.partitions.map { part =>
part.copy(values = castPartitionValuesToUserSchema(part.values))
})
case _ =>
PartitioningUtils.parsePartitions(
leafDirs,
PartitioningUtils.DEFAULT_PARTITION_NAME,
typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled(),
basePaths = basePaths)
}
}

private def prunePartitions(
predicates: Seq[Expression],
partitionSpec: PartitionSpec): Seq[PartitionDirectory] = {
val PartitionSpec(partitionColumns, partitions) = partitionSpec
Expand Down Expand Up @@ -365,11 +390,78 @@ class HDFSFileCatalog(
}
}

def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq
/**
* Contains a set of paths that are considered as the base dirs of the input datasets.
* The partitioning discovery logic will make sure it will stop when it reaches any
* base path. By default, the paths of the dataset provided by users will be base paths.
* For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path
* will be `/path/something=true/`, and the returned DataFrame will not contain a column of
* `something`. If users want to override the basePath. They can set `basePath` in the options
* to pass the new base path to the data source.
* For the above example, if the user-provided base path is `/path/`, the returned
* DataFrame will have the column of `something`.
*/
private def basePaths: Set[Path] = {
val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath)))
userDefinedBasePath.getOrElse {
// If the user does not provide basePath, we will just use paths.
paths.toSet
}.map { hdfsPath =>
// Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel).
val fs = hdfsPath.getFileSystem(hadoopConf)
hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
}
}
}


/**
* A file catalog that caches metadata gathered by scanning all the files present in `paths`
* recursively.
*
* @param parameters as set of options to control discovery
* @param paths a list of paths to scan
* @param partitionSchema an optional partition schema that will be use to provide types for the
* discovered partitions
*/
class HDFSFileCatalog(
Copy link
Contributor

Choose a reason for hiding this comment

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

existing: can you move the implementation into their own files (each separate). they aren't really interfaces.

sparkSession: SparkSession,
parameters: Map[String, String],
override val paths: Seq[Path],
partitionSchema: Option[StructType])
extends PartitioningAwareFileCatalog(sparkSession, parameters, partitionSchema) {

@volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _
@volatile private var cachedLeafDirToChildrenFiles: Map[Path, Array[FileStatus]] = _
@volatile private var cachedPartitionSpec: PartitionSpec = _

refresh()

override def partitionSpec(): PartitionSpec = {
if (cachedPartitionSpec == null) {
cachedPartitionSpec = inferPartitioning()
}
cachedPartitionSpec
}

override protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = {
cachedLeafFiles
}

override protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = {
cachedLeafDirToChildrenFiles
}

override def refresh(): Unit = {
val files = listLeafFiles(paths)
cachedLeafFiles =
new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f)
cachedLeafDirToChildrenFiles = files.toArray.groupBy(_.getPath.getParent)
cachedPartitionSpec = null
}

def getStatus(path: Path): Array[FileStatus] = leafDirToChildrenFiles(path)

private def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = {
protected def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = {
if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext)
} else {
Expand Down Expand Up @@ -415,73 +507,6 @@ class HDFSFileCatalog(
}
}

def inferPartitioning(schema: Option[StructType]): PartitionSpec = {
// We use leaf dirs containing data files to discover the schema.
val leafDirs = leafDirToChildrenFiles.keys.toSeq
schema match {
case Some(userProvidedSchema) if userProvidedSchema.nonEmpty =>
val spec = PartitioningUtils.parsePartitions(
leafDirs,
PartitioningUtils.DEFAULT_PARTITION_NAME,
typeInference = false,
basePaths = basePaths)

// Without auto inference, all of value in the `row` should be null or in StringType,
// we need to cast into the data type that user specified.
def castPartitionValuesToUserSchema(row: InternalRow) = {
InternalRow((0 until row.numFields).map { i =>
Cast(
Literal.create(row.getUTF8String(i), StringType),
userProvidedSchema.fields(i).dataType).eval()
}: _*)
}

PartitionSpec(userProvidedSchema, spec.partitions.map { part =>
part.copy(values = castPartitionValuesToUserSchema(part.values))
})
case _ =>
PartitioningUtils.parsePartitions(
leafDirs,
PartitioningUtils.DEFAULT_PARTITION_NAME,
typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled(),
basePaths = basePaths)
}
}

/**
* Contains a set of paths that are considered as the base dirs of the input datasets.
* The partitioning discovery logic will make sure it will stop when it reaches any
* base path. By default, the paths of the dataset provided by users will be base paths.
* For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path
* will be `/path/something=true/`, and the returned DataFrame will not contain a column of
* `something`. If users want to override the basePath. They can set `basePath` in the options
* to pass the new base path to the data source.
* For the above example, if the user-provided base path is `/path/`, the returned
* DataFrame will have the column of `something`.
*/
private def basePaths: Set[Path] = {
val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath)))
userDefinedBasePath.getOrElse {
// If the user does not provide basePath, we will just use paths.
paths.toSet
}.map { hdfsPath =>
// Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel).
val fs = hdfsPath.getFileSystem(hadoopConf)
hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
}
}

def refresh(): Unit = {
val files = listLeafFiles(paths)

leafFiles.clear()
leafDirToChildrenFiles.clear()

leafFiles ++= files.map(f => f.getPath -> f)
leafDirToChildrenFiles ++= files.toArray.groupBy(_.getPath.getParent)

cachedPartitionSpec = null
}

override def equals(other: Any): Boolean = other match {
case hdfs: HDFSFileCatalog => paths.toSet == hdfs.paths.toSet
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,42 +17,39 @@

package org.apache.spark.sql.execution.streaming

import scala.collection.mutable

import org.apache.hadoop.fs.{FileStatus, Path}

import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.datasources.{FileCatalog, Partition, PartitionSpec}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.execution.datasources._

class StreamFileCatalog(sparkSession: SparkSession, path: Path) extends FileCatalog with Logging {
val metadataDirectory = new Path(path, FileStreamSink.metadataDir)
logInfo(s"Reading streaming file log from $metadataDirectory")
val metadataLog = new FileStreamSinkLog(sparkSession, metadataDirectory.toUri.toString)
val fs = path.getFileSystem(sparkSession.sessionState.newHadoopConf())

override def paths: Seq[Path] = path :: Nil
class StreamFileCatalog(sparkSession: SparkSession, path: Path)
extends PartitioningAwareFileCatalog(sparkSession, Map.empty, None) {

private val metadataDirectory = new Path(path, FileStreamSink.metadataDir)
logInfo(s"Reading streaming file log from $metadataDirectory")
private val metadataLog = new FileStreamSinkLog(sparkSession, metadataDirectory.toUri.toString)
private val allFilesFromLog = metadataLog.allFiles().map(_.toFileStatus).filterNot(_.isDirectory)
private var cachedPartitionSpec: PartitionSpec = _

override def partitionSpec(): PartitionSpec = PartitionSpec(StructType(Nil), Nil)
override protected val leafFiles: mutable.LinkedHashMap[Path, FileStatus] = {
new mutable.LinkedHashMap ++= allFilesFromLog.map(f => f.getPath -> f)
}

/**
* Returns all valid files grouped into partitions when the data is partitioned. If the data is
* unpartitioned, this will return a single partition with not partition values.
*
* @param filters the filters used to prune which partitions are returned. These filters must
* only refer to partition columns and this method will only return files
* where these predicates are guaranteed to evaluate to `true`. Thus, these
* filters will not need to be evaluated again on the returned data.
*/
override def listFiles(filters: Seq[Expression]): Seq[Partition] =
Partition(InternalRow.empty, allFiles()) :: Nil
override protected val leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = {
allFilesFromLog.toArray.groupBy(_.getPath.getParent)
}

override def getStatus(path: Path): Array[FileStatus] = fs.listStatus(path)
override def paths: Seq[Path] = path :: Nil

override def refresh(): Unit = {}
override def refresh(): Unit = { }

override def allFiles(): Seq[FileStatus] = {
metadataLog.allFiles().map(_.toFileStatus)
override def partitionSpec(): PartitionSpec = {
if (cachedPartitionSpec == null) {
cachedPartitionSpec = inferPartitioning()
}
cachedPartitionSpec
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext {
}
}

test("FileStreamSink - partitioned writing and batch reading [IGNORES PARTITION COLUMN]") {
test("FileStreamSink - partitioned writing and batch reading") {
val inputData = MemoryStream[Int]
val ds = inputData.toDS()

Expand All @@ -171,11 +171,10 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext {
query.processAllAvailable()
}

// TODO (tdas): Test partition column can be read or not
val outputDf = sqlContext.read.parquet(outputDir)
checkDataset(
outputDf.as[Int],
1000, 2000, 3000)
outputDf.as[(Int, Int)],
(1000, 1), (2000, 2), (3000, 3))

} finally {
if (query != null) {
Expand Down