-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-25348][SQL] Data source for binary files #24354
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
Changes from all commits
42d1fc9
373af0f
a7aed42
55a6858
c3d4411
b711773
dda654a
aab4dcd
2b1780f
46a07e3
dd8e8c6
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,177 @@ | ||
| /* | ||
| * 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 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.{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 | ||
|
|
||
|
|
||
| /** | ||
| * The binary file data source. | ||
| * | ||
| * It reads binary files and converts each file into a single record that contains the raw content | ||
| * and metadata of the file. | ||
| * | ||
| * Example: | ||
| * {{{ | ||
| * // Scala | ||
| * val df = spark.read.format("binaryFile") | ||
| * .option("pathGlobFilter", "*.png") | ||
| * .load("/path/to/fileDir") | ||
| * | ||
| * // Java | ||
| * Dataset<Row> df = spark.read().format("binaryFile") | ||
| * .option("pathGlobFilter", "*.png") | ||
| * .load("/path/to/fileDir"); | ||
| * }}} | ||
| */ | ||
| class BinaryFileFormat extends FileFormat with DataSourceRegister { | ||
|
|
||
| override def inferSchema( | ||
| sparkSession: SparkSession, | ||
| options: Map[String, String], | ||
| files: Seq[FileStatus]): Option[StructType] = Some(BinaryFileFormat.schema) | ||
|
|
||
| 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 | ||
| } | ||
|
|
||
| override def shortName(): String = "binaryFile" | ||
WeichenXu123 marked this conversation as resolved.
Show resolved
Hide resolved
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| override protected def buildReader( | ||
| sparkSession: SparkSession, | ||
| dataSchema: StructType, | ||
| partitionSchema: StructType, | ||
| requiredSchema: StructType, | ||
| filters: Seq[Filter], | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. are we going to leverage the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
||
| 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 = fileStatus.getModificationTime() | ||
| val stream = fs.open(fsPath) | ||
WeichenXu123 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| val content = try { | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| ByteStreams.toByteArray(stream) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh, we can control it with |
||
| } finally { | ||
| Closeables.close(stream, true) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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) | ||
| } | ||
|
|
||
| // TODO: Add column pruning | ||
| // currently it still read the file content even if content column is not required. | ||
| val requiredColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.fromMillis(modificationTime), | ||
| length | ||
| ) | ||
| ) | ||
|
|
||
| Iterator(requiredColumns(internalRow)) | ||
| } else { | ||
| Iterator.empty | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| object BinaryFileFormat { | ||
|
|
||
| private val fileStatusSchema = StructType( | ||
| StructField("path", StringType, false) :: | ||
| StructField("modificationTime", TimestampType, false) :: | ||
| StructField("length", LongType, false) :: Nil) | ||
|
|
||
| /** | ||
| * Schema for the binary file data source. | ||
| * | ||
| * Schema: | ||
| * - content (BinaryType): The content of the file. | ||
| * - status (StructType): The status of the file. | ||
| * - path (StringType): The path of the file. | ||
| * - modificationTime (TimestampType): The modification time of the file. | ||
| * In some Hadoop FileSystem implementation, this might be unavailable and fallback to some | ||
| * default value. | ||
| * - length (LongType): The length of the file in bytes. | ||
| */ | ||
| val schema = StructType( | ||
| StructField("content", BinaryType, true) :: | ||
| StructField("status", fileStatusSchema, false) :: Nil) | ||
| } | ||
|
|
||
| class BinaryFileSourceOptions( | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not a big deal at all but let me leave a note before I forget. |
||
| @transient private val parameters: CaseInsensitiveMap[String]) extends Serializable { | ||
|
|
||
| def this(parameters: Map[String, String]) = this(CaseInsensitiveMap(parameters)) | ||
|
|
||
| /** | ||
| * An optional glob pattern to only include files with paths matching the pattern. | ||
| * The syntax follows [[org.apache.hadoop.fs.GlobFilter]]. | ||
| */ | ||
| val pathGlobFilter: Option[String] = parameters.get("pathGlobFilter") | ||
| } | ||
| 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 BinaryFileFormatSuite 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")) | ||
| } | ||
| } | ||
|
|
||
| } |
There was a problem hiding this comment.
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.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
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
binaryFilesbecause usually one binary is a minimal logical unit for arbitrary binary files.There was a problem hiding this comment.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if
isSplitablereturns 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.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@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.There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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)