Skip to content
Closed
Show file tree
Hide file tree
Changes from 9 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 @@ -8,3 +8,4 @@ org.apache.spark.sql.execution.datasources.v2.text.TextDataSourceV2
org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
org.apache.spark.sql.execution.streaming.sources.RateStreamProvider
org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider
org.apache.spark.sql.execution.datasources.binaryfile.BinaryFileFormat
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.datasources.binaryfile

import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.sql.types._

/**
* This "binaryFile" data source format implements Spark SQL data source API for loading binary
* file data as `DataFrame`.
*
Copy link
Contributor

Choose a reason for hiding this comment

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

Please also document how to control the input partition size. cc: @cloud-fan

* The loaded `DataFrame` has two columns, the schema is:
* - content: `BinaryType` (binary data of the file content)
* - status: `StructType` (the file status information)
*
* The schema of "status" column described above is:
* - path: `StringType` (the file path)
* - modificationTime: `TimestampType` (last modification time of the file, on some FS
* implementation, this might be not available
* and fallback to some default value.)
* - length: `LongType` (the file length)
*
* To use binary file data source, you need to set "binaryFile" as the format in `DataFrameReader`
* and optionally specify the data source options, available options include:
* - pathGlobFilter: Only include files with path matching the glob pattern.
* The glob pattern keeps the same behavior with Hadoop API
* `org.apache.hadoop.fs.FileSystem.globStatus(pathPattern)`
*
* In order to control the partition size, we can set spark sql configuration
* `spark.sql.files.maxPartitionBytes` and `spark.sql.files.openCostInBytes`.
*
* Example:
* {{{
* // Scala
* val df = spark.read.format("binaryFile")
* .option("pathGlobFilter", "*.txt")
Copy link
Member

@gengliangwang gengliangwang Apr 16, 2019

Choose a reason for hiding this comment

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

Nit: how about changing the extension name "*.txt" in the example, e.g. *.png or *.jpg

* .load("path/to/fileDir")
*
* // Java
* Dataset<Row> df = spark.read().format("binaryFile")
* .option("pathGlobFilter", "*.txt")
* .load("path/to/fileDir");
* }}}
*
* @note This binary file data source does not support saving dataframe to binary files.
* @note This class is public for documentation purpose. Please don't use this class directly.
* Rather, use the data source API as illustrated above.
*/
@Experimental
@Since("3.0.0")
class BinaryFileDataSource private() {}

object BinaryFileDataSource {

private val fileStatusSchema = StructType(
StructField("path", StringType, false) ::
StructField("modificationTime", TimestampType, false) ::
StructField("length", LongType, false) :: Nil)

/**
* The schema of the dataframe returned by binaryFile data source.
* See doc in `BinaryFileDataSource`
*/
val binaryFileSchema = StructType(
StructField("content", BinaryType, false)::
Copy link
Contributor

Choose a reason for hiding this comment

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

  • space before ::
  • just a note: we might keep this column nullable in case to handle potential I/O failures

StructField("status", fileStatusSchema, false) :: Nil)

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.datasources.binaryfile

import java.sql.Timestamp

import com.google.common.io.{ByteStreams, Closeables}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path}
import org.apache.hadoop.mapreduce.Job

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat, OutputWriterFactory, PartitionedFile}
import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.SerializableConfiguration


private[binaryfile] class BinaryFileFormat extends FileFormat with DataSourceRegister {
Copy link
Member

Choose a reason for hiding this comment

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

As per https://issues.apache.org/jira/browse/SPARK-16964, I think we can remove private[binaryfile]


override def inferSchema(
sparkSession: SparkSession,
options: Map[String, String],
files: Seq[FileStatus]): Option[StructType] = Some(BinaryFileDataSource.binaryFileSchema)

override def prepareWrite(
sparkSession: SparkSession,
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
throw new UnsupportedOperationException("Write is not supported for binary file data source")
}

override def isSplitable(
sparkSession: SparkSession,
options: Map[String, String],
path: Path): Boolean = {
false
Copy link
Contributor

Choose a reason for hiding this comment

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

are we sure about this? Always return false means one file one RDD partition.

Copy link
Member

@HyukjinKwon HyukjinKwon Apr 14, 2019

Choose a reason for hiding this comment

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

I don't think binary partitions should be splittable like binaryFiles because usually one binary is a minimal logical unit for arbitrary binary files.

Copy link
Contributor

Choose a reason for hiding this comment

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

@cloud-fan Does it mean that the file itself cannot be split into multiple parts? It shouldn't lead to one file per partition.

Copy link
Contributor

Choose a reason for hiding this comment

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

if isSplitable returns false, then Spark can only read the entire file with a single thread, so it's one file per partition.

The file splitting is actually very complicated. For example, the text format splits the file w.r.t. the line boundary. A line of text will not be split into multiple partitions. I'm not sure how to define the file splitting logic for binary files.

Copy link
Member

@HyukjinKwon HyukjinKwon Apr 16, 2019

Choose a reason for hiding this comment

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

It shouldn't lead to one file per partition.

@mengxr, do you mean one binary file should be split into multiple parts? In that case, the splitting rule should be defined and fixed so that end users can process it and use it. If users are not aware of the rule to split up, there wouldn't be a way for users to use it (for instance image).

I thought this will be implemented by maxBytesPerPartition, and by default one file per one partition.

Copy link
Contributor

Choose a reason for hiding this comment

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

@cloud-fan Why single thread leads to one file per partition? One partition can still have multiple files, but one file cannot be split into multiple records.

@HyukjinKwon We don't want to split a file into parts.

Copy link
Contributor

Choose a reason for hiding this comment

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

to be clear: one file per file partition. It's still possible that one RDD partition contains many files.

Copy link
Member

Choose a reason for hiding this comment

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

(Oops, I was confused that we were talking about one partition that has multiple parts)

}

override def shortName(): String = "binaryFile"

override protected def buildReader(
sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
Copy link
Contributor

@cloud-fan cloud-fan Apr 14, 2019

Choose a reason for hiding this comment

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

are we going to leverage the filters here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I can put it in later PR.

options: Map[String, String],
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {

val broadcastedHadoopConf =
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))

val binaryFileSourceOptions = new BinaryFileSourceOptions(options)

val pathGlobPattern = binaryFileSourceOptions.pathGlobFilter

(file: PartitionedFile) => {
val path = file.filePath
val fsPath = new Path(path)

// TODO: Improve performance here: each file will recompile the glob pattern here.
Copy link
Member

Choose a reason for hiding this comment

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

I think we should make it a general option, which can be applied in all data sources. Also we should pass the option to FileIndex, so that Spark can split file partition more precisely.
We can have a follow-up PR for this.

val globFilter = pathGlobPattern.map(new GlobFilter(_))
if (!globFilter.isDefined || globFilter.get.accept(fsPath)) {
val fs = fsPath.getFileSystem(broadcastedHadoopConf.value.value)
val fileStatus = fs.getFileStatus(fsPath)
val length = fileStatus.getLen()
val modificationTime = new Timestamp(fileStatus.getModificationTime())
val stream = fs.open(fsPath)

val content = try {
ByteStreams.toByteArray(stream)
Copy link
Member

Choose a reason for hiding this comment

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

If I remember correctly, the usual behavior in Spark is not to throw an exception but prefers null value. At this point, should we assign content null value instead of throwing exception?

Copy link
Member

Choose a reason for hiding this comment

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

Oh, we can control it with ignoreCorruptFiles.

} finally {
Closeables.close(stream, true)
Copy link
Member

Choose a reason for hiding this comment

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

Related to above comment, should we not propagate IO exceptions?

}

val fullOutput = dataSchema.map { f =>
AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()
}
val requiredOutput = fullOutput.filter { a =>
requiredSchema.fieldNames.contains(a.name)
}

val requiredColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput)
Copy link
Contributor

Choose a reason for hiding this comment

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

this does not help the performance. We still read the file content even if content column is not required.

Copy link
Contributor

Choose a reason for hiding this comment

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

This is OK for now, maybe we can leave a TODO and implement the real column pruning in the future.


val internalRow = InternalRow(
content,
InternalRow(
UTF8String.fromString(path),
DateTimeUtils.fromJavaTimestamp(modificationTime),
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it more straightforward to use fromMillis?

Copy link
Contributor

Choose a reason for hiding this comment

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

yes, we should use DateTimeUtils.fromMillis(fileStatus.getModificationTime())

length
)
)

Iterator(requiredColumns(internalRow))
} else {
Iterator.empty
}
}
}
}

private[binaryfile] class BinaryFileSourceOptions(
Copy link
Member

Choose a reason for hiding this comment

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

Remove private[binaryfile] here as well.

@transient private val parameters: CaseInsensitiveMap[String]) extends Serializable {

def this(parameters: Map[String, String]) = this(CaseInsensitiveMap(parameters))

/**
* only include files with path matching the glob pattern.
*/
val pathGlobFilter: Option[String] = {
val filter = parameters.getOrElse("pathGlobFilter", null)
Copy link
Contributor

Choose a reason for hiding this comment

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

Just parameters.get("pathGlobFilter") should work

if (filter != null) Some(filter) else None
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.datasources.binaryfile

import java.io.File
import java.nio.file.{Files, StandardOpenOption}
import java.sql.Timestamp

import scala.collection.JavaConverters._

import com.google.common.io.{ByteStreams, Closeables}
import org.apache.hadoop.fs.{FileSystem, GlobFilter, Path}

import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
import org.apache.spark.util.Utils

class BinaryFileSuite extends QueryTest with SharedSQLContext with SQLTestUtils {

private var testDir: String = _

private var fsTestDir: Path = _

private var fs: FileSystem = _

override def beforeAll(): Unit = {
super.beforeAll()

testDir = Utils.createTempDir().getAbsolutePath
fsTestDir = new Path(testDir)
fs = fsTestDir.getFileSystem(sparkContext.hadoopConfiguration)

val year2014Dir = new File(testDir, "year=2014")
year2014Dir.mkdir()
val year2015Dir = new File(testDir, "year=2015")
year2015Dir.mkdir()

Files.write(
new File(year2014Dir, "data.txt").toPath,
Seq("2014-test").asJava,
StandardOpenOption.CREATE, StandardOpenOption.WRITE
)
Files.write(
new File(year2014Dir, "data2.bin").toPath,
"2014-test-bin".getBytes,
StandardOpenOption.CREATE, StandardOpenOption.WRITE
)

Files.write(
new File(year2015Dir, "bool.csv").toPath,
Seq("bool", "True", "False", "true").asJava,
StandardOpenOption.CREATE, StandardOpenOption.WRITE
)
Files.write(
new File(year2015Dir, "data.txt").toPath,
"2015-test".getBytes,
StandardOpenOption.CREATE, StandardOpenOption.WRITE
)
}

def testBinaryFileDataSource(pathGlobFilter: String): Unit = {
val resultDF = spark.read.format("binaryFile")
.option("pathGlobFilter", pathGlobFilter)
.load(testDir)
.select(
col("status.path"),
col("status.modificationTime"),
col("status.length"),
col("content"),
col("year") // this is a partition column
)

val expectedRowSet = new collection.mutable.HashSet[Row]()

val globFilter = new GlobFilter(pathGlobFilter)
for (partitionDirStatus <- fs.listStatus(fsTestDir)) {
val dirPath = partitionDirStatus.getPath

val partitionName = dirPath.getName.split("=")(1)
val year = partitionName.toInt // partition column "year" value which is `Int` type

for (fileStatus <- fs.listStatus(dirPath)) {
if (globFilter.accept(fileStatus.getPath)) {
val fpath = fileStatus.getPath.toString.replace("file:/", "file:///")
val flen = fileStatus.getLen
val modificationTime = new Timestamp(fileStatus.getModificationTime)

val fcontent = {
val stream = fs.open(fileStatus.getPath)
val content = try {
ByteStreams.toByteArray(stream)
} finally {
Closeables.close(stream, true)
}
content
}

val row = Row(fpath, modificationTime, flen, fcontent, year)
expectedRowSet.add(row)
}
}
}

checkAnswer(resultDF, expectedRowSet.toSeq)
}

test("binary file data source test") {
testBinaryFileDataSource(pathGlobFilter = "*.*")
testBinaryFileDataSource(pathGlobFilter = "*.bin")
testBinaryFileDataSource(pathGlobFilter = "*.txt")
testBinaryFileDataSource(pathGlobFilter = "*.{txt,csv}")
testBinaryFileDataSource(pathGlobFilter = "*.json")
}

test ("binary file data source do not support write operation") {
val df = spark.read.format("binaryFile").load(testDir)
withTempDir { tmpDir =>
val thrown = intercept[UnsupportedOperationException] {
df.write
.format("binaryFile")
.save(tmpDir + "/test_save")
}
assert(thrown.getMessage.contains("Write is not supported for binary file data source"))
}
}

}