-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-2759][CORE] Generic Binary File Support in Spark #1658
Changes from all 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, PortableDataStream, WholeTextFileInputFormat, 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,52 @@ class SparkContext(config: SparkConf) extends Logging { | |
minPartitions).setName(path) | ||
} | ||
|
||
|
||
/** | ||
* Get an RDD for a Hadoop-readable dataset as PortableDataStream 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. | ||
*/ | ||
@DeveloperApi | ||
def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): | ||
RDD[(String, PortableDataStream)] = { | ||
val job = new NewHadoopJob(hadoopConfiguration) | ||
NewFileInputFormat.addInputPath(job, new Path(path)) | ||
val updateConf = job.getConfiguration | ||
new BinaryFileRDD( | ||
this, | ||
classOf[StreamInputFormat], | ||
classOf[String], | ||
classOf[PortableDataStream], | ||
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 | ||
* @param recordLength The length at which to split the records | ||
* @return An RDD of data with values, RDD[(Array[Byte])] | ||
*/ | ||
def binaryRecords(path: String, recordLength: Int, | ||
conf: Configuration = hadoopConfiguration): RDD[Array[Byte]] = { | ||
conf.setInt("recordLength",recordLength) | ||
val br = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path, | ||
classOf[FixedLengthBinaryInputFormat], | ||
classOf[LongWritable], | ||
classOf[BytesWritable], | ||
conf=conf) | ||
val data = br.map{ case (k, v) => v.getBytes} | ||
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 turns out that Using Is the use of |
||
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 +1254,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 |
---|---|---|
@@ -0,0 +1,87 @@ | ||
/* | ||
* 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. | ||
*/ | ||
|
||
private[spark] object FixedLengthBinaryInputFormat { | ||
/** | ||
* 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 | ||
} | ||
} | ||
|
||
private[spark] 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 | ||
if (defaultSize < recordLength) { | ||
recordLength.toLong | ||
} else { | ||
(Math.floor(defaultSize / recordLength) * recordLength).toLong | ||
} | ||
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 | ||
} | ||
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. Nit: the indent is slightly wrong here, should look like
|
||
var recordLength = -1 | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,129 @@ | ||
/* | ||
* 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 java.io.IOException | ||
|
||
import org.apache.hadoop.fs.FSDataInputStream | ||
import org.apache.hadoop.io.compress.CompressionCodecFactory | ||
import org.apache.hadoop.io.{BytesWritable, LongWritable} | ||
import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} | ||
import org.apache.hadoop.mapreduce.lib.input.FileSplit | ||
|
||
/** | ||
* | ||
* FixedLengthBinaryRecordReader is returned by FixedLengthBinaryInputFormat. | ||
* It uses the record length set in FixedLengthBinaryInputFormat to | ||
* read one record at a time from the given InputSplit. | ||
* | ||
* Each call to nextKeyValue() updates the LongWritable KEY and BytesWritable VALUE. | ||
* | ||
* KEY = record index (Long) | ||
* VALUE = the record itself (BytesWritable) | ||
* | ||
*/ | ||
private[spark] class FixedLengthBinaryRecordReader | ||
extends RecordReader[LongWritable, BytesWritable] { | ||
|
||
override def close() { | ||
if (fileInputStream != null) { | ||
fileInputStream.close() | ||
} | ||
} | ||
|
||
override def getCurrentKey: LongWritable = { | ||
recordKey | ||
} | ||
|
||
override def getCurrentValue: BytesWritable = { | ||
recordValue | ||
} | ||
|
||
override def getProgress: Float = { | ||
splitStart match { | ||
case x if x == splitEnd => 0.0.toFloat | ||
case _ => Math.min( | ||
((currentPosition - splitStart) / (splitEnd - splitStart)).toFloat, 1.0 | ||
).toFloat | ||
} | ||
} | ||
|
||
override def initialize(inputSplit: InputSplit, context: TaskAttemptContext) { | ||
|
||
// the file input | ||
val fileSplit = inputSplit.asInstanceOf[FileSplit] | ||
|
||
// the byte position this fileSplit starts at | ||
splitStart = fileSplit.getStart | ||
|
||
// splitEnd byte marker that the fileSplit ends at | ||
splitEnd = splitStart + fileSplit.getLength | ||
|
||
// the actual file we will be reading from | ||
val file = fileSplit.getPath | ||
// job configuration | ||
val job = context.getConfiguration | ||
// check compression | ||
val codec = new CompressionCodecFactory(job).getCodec(file) | ||
if (codec != null) { | ||
throw new IOException("FixedLengthRecordReader does not support reading compressed files") | ||
} | ||
// get the record length | ||
recordLength = FixedLengthBinaryInputFormat.getRecordLength(context) | ||
// get the filesystem | ||
val fs = file.getFileSystem(job) | ||
// open the File | ||
fileInputStream = fs.open(file) | ||
// seek to the splitStart position | ||
fileInputStream.seek(splitStart) | ||
// set our current position | ||
currentPosition = splitStart | ||
} | ||
|
||
override def nextKeyValue(): Boolean = { | ||
if (recordKey == null) { | ||
recordKey = new LongWritable() | ||
} | ||
// the key is a linear index of the record, given by the | ||
// position the record starts divided by the record length | ||
recordKey.set(currentPosition / recordLength) | ||
// the recordValue to place the bytes into | ||
if (recordValue == null) { | ||
recordValue = new BytesWritable(new Array[Byte](recordLength)) | ||
} | ||
// read a record if the currentPosition is less than the split end | ||
if (currentPosition < splitEnd) { | ||
// setup a buffer to store the record | ||
val buffer = recordValue.getBytes | ||
fileInputStream.read(buffer, 0, recordLength) | ||
// update our current position | ||
currentPosition = currentPosition + recordLength | ||
// return true | ||
return true | ||
} | ||
false | ||
} | ||
var splitStart: Long = 0L | ||
var splitEnd: Long = 0L | ||
var currentPosition: Long = 0L | ||
var recordLength: Int = 0 | ||
var fileInputStream: FSDataInputStream = null | ||
var recordKey: LongWritable = null | ||
var recordValue: BytesWritable = null | ||
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 these at the top of the class to make it a bit easier to read. Also they can be private. |
||
|
||
} |
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.
There's no need to have
@DeveloperApi
on this, as PortableDataStream is something we want to support.