-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-2759][CORE] Generic Binary File Support in Spark #1658
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 7 commits
81c5f12
84035f1
1cfa38a
1622935
eacfaa6
f4841dc
edf5829
9a313d5
df8e528
bc5c0b9
f032bc0
5deb79e
12e7be1
441f79a
a01c9cf
932a206
238c83c
19812a8
4163e38
0588737
b348ce1
c27a8f1
49174d9
a32fef7
92bda0d
8ac288b
7b9d181
6379be4
359a096
3c49a30
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 |
|---|---|---|
|
|
@@ -40,7 +40,7 @@ import org.apache.mesos.MesosNativeLibrary | |
| import org.apache.spark.annotation.{DeveloperApi, Experimental} | ||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} | ||
| import org.apache.spark.input.WholeTextFileInputFormat | ||
| import org.apache.spark.input.{StreamInputFormat, StreamFileInputFormat, WholeTextFileInputFormat, ByteInputFormat, FixedLengthBinaryInputFormat} | ||
| import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} | ||
| import org.apache.spark.rdd._ | ||
| import org.apache.spark.scheduler._ | ||
|
|
@@ -289,7 +289,7 @@ class SparkContext(config: SparkConf) extends Logging { | |
| value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { | ||
| executorEnvs(envKey) = value | ||
| } | ||
| Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => | ||
| Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => | ||
| executorEnvs("SPARK_PREPEND_CLASSES") = v | ||
| } | ||
| // The Mesos scheduler backend relies on this environment variable to set executor memory. | ||
|
|
@@ -510,6 +510,67 @@ class SparkContext(config: SparkConf) extends Logging { | |
| minPartitions).setName(path) | ||
| } | ||
|
|
||
| /** | ||
| * Get an RDD for a Hadoop-readable dataset as byte-streams for each file | ||
| * (useful for binary data) | ||
| * | ||
| * @param minPartitions A suggestion value of the minimal splitting number for input data. | ||
| * | ||
| * @note Small files are preferred, large file is also allowable, but may cause bad performance. | ||
| */ | ||
| def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): | ||
| RDD[(String, Array[Byte])] = { | ||
| val job = new NewHadoopJob(hadoopConfiguration) | ||
| NewFileInputFormat.addInputPath(job, new Path(path)) | ||
| val updateConf = job.getConfiguration | ||
| new RawFileRDD( | ||
| this, | ||
| classOf[ByteInputFormat], | ||
| classOf[String], | ||
| classOf[Array[Byte]], | ||
| updateConf, | ||
| minPartitions).setName(path) | ||
| } | ||
|
|
||
| /** | ||
| * Get an RDD for a Hadoop-readable dataset as DataInputStreams for each file | ||
| * (useful for binary data) | ||
| * | ||
| * | ||
| * @param minPartitions A suggestion value of the minimal splitting number for input data. | ||
| * | ||
| * @note Care must be taken to close the files afterwards | ||
|
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. It is a bit unfortunate that users have to close the streams by hand. If you want to get around this, you can create a custom RDD wrapping around the HadoopRDD, whose compute() method can add a cleanup hook to its TaskContext to close the stream. Take a look at TaskContext.addOnCompleteCallback().
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. Hey Kevin, is this |
||
| * @note Small files are preferred, large file is also allowable, but may cause bad performance. | ||
| */ | ||
| @DeveloperApi | ||
|
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. There's no need to have |
||
| def dataStreamFiles(path: String, minPartitions: Int = defaultMinPartitions): | ||
| RDD[(String, DataInputStream)] = { | ||
| val job = new NewHadoopJob(hadoopConfiguration) | ||
| NewFileInputFormat.addInputPath(job, new Path(path)) | ||
| val updateConf = job.getConfiguration | ||
| new RawFileRDD( | ||
| this, | ||
| classOf[StreamInputFormat], | ||
| classOf[String], | ||
| classOf[DataInputStream], | ||
| updateConf, | ||
| minPartitions).setName(path) | ||
| } | ||
|
|
||
| /** | ||
| * Load data from a flat binary file, assuming each record is a set of numbers | ||
| * with the specified numerical format (see ByteBuffer), and the number of | ||
| * bytes per record is constant (see FixedLengthBinaryInputFormat) | ||
| * | ||
| * @param path Directory to the input data files | ||
| * @return An RDD of data with values, RDD[(Array[Byte])] | ||
| */ | ||
| def fixedLengthBinaryFiles(path: String): RDD[Array[Byte]] = { | ||
|
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. This has been taken almost directly from |
||
| val lines = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path) | ||
| val data = lines.map{ case (k, v) => v.getBytes} | ||
| data | ||
| } | ||
|
|
||
| /** | ||
| * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other | ||
| * necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable), | ||
|
|
@@ -1208,7 +1269,7 @@ class SparkContext(config: SparkConf) extends Logging { | |
| * If <tt>checkSerializable</tt> is set, <tt>clean</tt> will also proactively | ||
| * check to see if <tt>f</tt> is serializable and throw a <tt>SparkException</tt> | ||
| * if not. | ||
| * | ||
| * | ||
| * @param f the closure to clean | ||
| * @param checkSerializable whether or not to immediately check <tt>f</tt> for serializability | ||
| * @throws <tt>SparkException<tt> if <tt>checkSerializable</tt> is set but <tt>f</tt> is not | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,6 +20,11 @@ package org.apache.spark.api.java | |
| import java.util | ||
| import java.util.{Map => JMap} | ||
|
|
||
| import java.io.DataInputStream | ||
|
|
||
| import org.apache.hadoop.io.{BytesWritable, LongWritable} | ||
| import org.apache.spark.input.FixedLengthBinaryInputFormat | ||
|
|
||
| import scala.collection.JavaConversions | ||
| import scala.collection.JavaConversions._ | ||
| import scala.language.implicitConversions | ||
|
|
@@ -180,6 +185,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork | |
| def textFile(path: String, minPartitions: Int): JavaRDD[String] = | ||
| sc.textFile(path, minPartitions) | ||
|
|
||
|
|
||
|
|
||
| /** | ||
| * Read a directory of text files from HDFS, a local file system (available on all nodes), or any | ||
| * Hadoop-supported file system URI. Each file is read as a single record and returned in a | ||
|
|
@@ -220,6 +227,82 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork | |
| def wholeTextFiles(path: String): JavaPairRDD[String, String] = | ||
| new JavaPairRDD(sc.wholeTextFiles(path)) | ||
|
|
||
| /** | ||
| * Read a directory of binary files from HDFS, a local file system (available on all nodes), | ||
| * or any Hadoop-supported file system URI as a byte array. Each file is read as a single | ||
| * record and returned in a key-value pair, where the key is the path of each file, | ||
| * the value is the content of each file. | ||
| * | ||
| * <p> For example, if you have the following files: | ||
| * {{{ | ||
| * hdfs://a-hdfs-path/part-00000 | ||
| * hdfs://a-hdfs-path/part-00001 | ||
| * ... | ||
| * hdfs://a-hdfs-path/part-nnnnn | ||
| * }}} | ||
| * | ||
| * Do | ||
| * `JavaPairRDD<String, byte[]> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, | ||
| * | ||
| * <p> then `rdd` contains | ||
| * {{{ | ||
| * (a-hdfs-path/part-00000, its content) | ||
| * (a-hdfs-path/part-00001, its content) | ||
| * ... | ||
| * (a-hdfs-path/part-nnnnn, its content) | ||
| * }}} | ||
| * | ||
| * @note Small files are preferred, large file is also allowable, but may cause bad performance. | ||
| * | ||
| * @param minPartitions A suggestion value of the minimal splitting number for input data. | ||
| */ | ||
| def dataStreamFiles(path: String, minPartitions: Int = defaultMinPartitions): | ||
| JavaPairRDD[String,DataInputStream] = new JavaPairRDD(sc.dataStreamFiles(path,minPartitions)) | ||
|
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. Put spaces after the commas and make sure the indentation matches our style guide (here it should be indented by two spaces after the "def"). |
||
|
|
||
| /** | ||
| * Read a directory of files as DataInputStream from HDFS, | ||
| * a local file system (available on all nodes), or any Hadoop-supported file system URI | ||
| * as a byte array. Each file is read as a single record and returned in a | ||
| * key-value pair, where the key is the path of each file, the value is the content of each. | ||
| * | ||
| * <p> For example, if you have the following files: | ||
| * {{{ | ||
| * hdfs://a-hdfs-path/part-00000 | ||
| * hdfs://a-hdfs-path/part-00001 | ||
| * ... | ||
| * hdfs://a-hdfs-path/part-nnnnn | ||
| * }}} | ||
| * | ||
| * Do | ||
| * `JavaPairRDD<String,DataInputStream> rdd = sparkContext.binaryFiles("hdfs://a-hdfs-path")`, | ||
| * | ||
| * <p> then `rdd` contains | ||
| * {{{ | ||
| * (a-hdfs-path/part-00000, its content) | ||
| * (a-hdfs-path/part-00001, its content) | ||
| * ... | ||
| * (a-hdfs-path/part-nnnnn, its content) | ||
| * }}} | ||
| * | ||
| * @note Small files are preferred, large file is also allowable, but may cause bad performance. | ||
| * | ||
| * @param minPartitions A suggestion value of the minimal splitting number for input data. | ||
| */ | ||
| def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): | ||
| JavaPairRDD[String,Array[Byte]] = new JavaPairRDD(sc.binaryFiles(path,minPartitions)) | ||
|
|
||
| /** | ||
| * Load data from a flat binary file, assuming each record is a set of numbers | ||
| * with the specified numerical format (see ByteBuffer), and the number of | ||
| * bytes per record is constant (see FixedLengthBinaryInputFormat) | ||
| * | ||
| * @param path Directory to the input data files | ||
| * @return An RDD of data with values, JavaRDD[(Array[Byte])] | ||
| */ | ||
| def fixedLengthBinaryFiles(path: String): JavaRDD[Array[Byte]] = { | ||
| new JavaRDD(sc.fixedLengthBinaryFiles(path)) | ||
| } | ||
|
|
||
| /** Get an RDD for a Hadoop SequenceFile with given key and value types. | ||
| * | ||
| * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,95 @@ | ||
| /* | ||
| * 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.input | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.io.{BytesWritable, LongWritable} | ||
| import org.apache.hadoop.mapreduce.lib.input.FileInputFormat | ||
| import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} | ||
|
|
||
| /** | ||
| * Custom Input Format for reading and splitting flat binary files that contain records, | ||
| * each of which are a fixed size in bytes. The fixed record size is specified through | ||
| * a parameter recordLength in the Hadoop configuration. | ||
| */ | ||
|
|
||
| object FixedLengthBinaryInputFormat { | ||
|
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. All of these objects should be marked |
||
|
|
||
| /** | ||
| * This function retrieves the recordLength by checking the configuration parameter | ||
| * | ||
| */ | ||
| def getRecordLength(context: JobContext): Int = { | ||
|
|
||
| // retrieve record length from configuration | ||
| context.getConfiguration.get("recordLength").toInt | ||
| } | ||
|
|
||
| } | ||
|
|
||
| class FixedLengthBinaryInputFormat extends FileInputFormat[LongWritable, BytesWritable] { | ||
|
|
||
|
|
||
| /** | ||
| * Override of isSplitable to ensure initial computation of the record length | ||
| */ | ||
| override def isSplitable(context: JobContext, filename: Path): Boolean = { | ||
|
|
||
| if (recordLength == -1) { | ||
| recordLength = FixedLengthBinaryInputFormat.getRecordLength(context) | ||
| } | ||
| if (recordLength <= 0) { | ||
| println("record length is less than 0, file cannot be split") | ||
| false | ||
| } else { | ||
| true | ||
| } | ||
|
|
||
| } | ||
|
|
||
| /** | ||
| * This input format overrides computeSplitSize() to make sure that each split | ||
| * only contains full records. Each InputSplit passed to FixedLengthBinaryRecordReader | ||
| * will start at the first byte of a record, and the last byte will the last byte of a record. | ||
| */ | ||
| override def computeSplitSize(blockSize: Long, minSize: Long, maxSize: Long): Long = { | ||
|
|
||
| val defaultSize = super.computeSplitSize(blockSize, minSize, maxSize) | ||
|
|
||
| // If the default size is less than the length of a record, make it equal to it | ||
| // Otherwise, make sure the split size is as close to possible as the default size, | ||
| // but still contains a complete set of records, with the first record | ||
| // starting at the first byte in the split and the last record ending with the last byte | ||
|
|
||
| defaultSize match { | ||
| case x if x < recordLength => recordLength.toLong | ||
| case _ => (Math.floor(defaultSize / recordLength) * recordLength).toLong | ||
| } | ||
|
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. Probably clearer to write this as |
||
| } | ||
|
|
||
| /** | ||
| * Create a FixedLengthBinaryRecordReader | ||
| */ | ||
| override def createRecordReader(split: InputSplit, context: TaskAttemptContext): | ||
| RecordReader[LongWritable, BytesWritable] = { | ||
| new FixedLengthBinaryRecordReader | ||
| } | ||
|
||
|
|
||
| var recordLength = -1 | ||
|
|
||
| } | ||
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.
What if we just added a toArray to PortableDataStream, and had only one method for reading these? Then you could do
sc.binaryFiles(...).map(_.toArray)if you want to get byte arrays. Or would this cause a regression?Basically my suggestion is to have binaryFiles, which will return an RDD of PortableDataStream, and binaryRecords, which will return an RDD of byte arrays of the same length (since I imagine there's no point streaming a record).