Skip to content
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

Closed
wants to merge 30 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
81c5f12
Merge pull request #1 from apache/master
kmader Jul 30, 2014
84035f1
adding binary and byte file support spark
kmader Jul 30, 2014
1cfa38a
added apache headers, added datainputstream directly as an output opt…
kmader Jul 31, 2014
1622935
changing the line lengths to make jenkins happy
kmader Jul 31, 2014
eacfaa6
Added FixedLengthBinaryInputFormat and RecordReader from freeman-lab …
kmader Aug 13, 2014
f4841dc
un-optimizing imports, silly intellij
kmader Aug 13, 2014
edf5829
fixing line lengths, adding new lines
kmader Aug 13, 2014
9a313d5
making classes that needn't be public private, adding automatic file …
kmader Aug 14, 2014
df8e528
fixed line lengths and changed java test
kmader Aug 14, 2014
bc5c0b9
made minor stylistic adjustments from mateiz
kmader Aug 14, 2014
f032bc0
fixed bug in path name, renamed tests
kmader Aug 14, 2014
5deb79e
added new portabledatastream to code so that it can be serialized cor…
kmader Sep 7, 2014
12e7be1
removing imglib from maven (definitely not ready yet)
kmader Sep 7, 2014
441f79a
fixed a few small comments and dependency
kmader Sep 7, 2014
a01c9cf
Update RawFileInput.scala
kmader Sep 16, 2014
932a206
Update RawFileInput.scala
kmader Sep 16, 2014
238c83c
fixed several scala-style issues, changed structure of binaryFiles, r…
kmader Oct 1, 2014
19812a8
Fixed the serialization issue with PortableDataStream since neither C…
kmader Oct 1, 2014
4163e38
fixing line length and output from FSDataInputStream to DataInputStre…
kmader Oct 1, 2014
0588737
filename check in "binary file input as byte array" test now ignores …
kmader Oct 1, 2014
b348ce1
fixed order in check (prefix only appears on jenkins not when I run u…
kmader Oct 1, 2014
c27a8f1
jenkins crashed before running anything last time, so making minor ch…
kmader Oct 1, 2014
49174d9
removed unneeded classes added DeveloperApi note to portabledatastrea…
kmader Oct 2, 2014
a32fef7
removed unneeded classes added DeveloperApi note to portabledatastrea…
kmader Oct 2, 2014
92bda0d
added new tests, renamed files, fixed several of the javaapi function…
kmader Oct 20, 2014
8ac288b
fixed a single slightly over 100 character line
kmader Oct 21, 2014
7b9d181
removing developer API, cleaning up imports
kmader Oct 21, 2014
6379be4
reorganizing code
kmader Oct 21, 2014
359a096
making the final corrections suggested by @mateiz and renaming a few …
kmader Oct 30, 2014
3c49a30
fixing wholetextfileinput to it has the same setMinPartitions functio…
kmader Oct 30, 2014
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
52 changes: 49 additions & 3 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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
Copy link
Contributor

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.

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}
Copy link
Contributor

Choose a reason for hiding this comment

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

It turns out that getBytes returns a padded byte array, so I think you may need to copy / slice out the subarray with the data using v.getLength; see HADOOP-6298: "BytesWritable#getBytes is a bad name that leads to programming mistakes" for more details.

Using getBytes without getLength has caused bugs in Spark in the past: #2712.

Is the use of getBytes in this patch a bug? Or is it somehow safe due to our use of FixedLengthBinaryInputFormat? If it is somehow safe, we should have a comment which explains this so that readers who know about the getBytes issue aren't confused (or better yet, an assert that getBytes returns an array of the expected length).

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),
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.{PortableDataStream, FixedLengthBinaryInputFormat}

import scala.collection.JavaConversions
import scala.collection.JavaConversions._
import scala.language.implicitConversions
Expand All @@ -31,7 +36,7 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}

import org.apache.spark._
import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam}
import org.apache.spark.SparkContext._
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.{EmptyRDD, RDD}
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -220,6 +227,54 @@ 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 binaryFiles(path: String, minPartitions: Int):
JavaPairRDD[String,PortableDataStream] = new JavaPairRDD(sc.binaryFiles(path,minPartitions))

def binaryFiles(path: String):
JavaPairRDD[String,PortableDataStream] =
new JavaPairRDD(sc.binaryFiles(path,defaultMinPartitions))

/**
* 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 binaryRecords(path: String,recordLength: Int): JavaRDD[Array[Byte]] = {
new JavaRDD(sc.binaryRecords(path,recordLength))
}

/** 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
Expand Down
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
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Probably clearer to write this as

if (defaultSize < recordLength) {
  recordLenght.toLong
} else {
  (Math.floor(defaultSize / recordLength) * recordLength).toLong
}

}

/**
* Create a FixedLengthBinaryRecordReader
*/
override def createRecordReader(split: InputSplit, context: TaskAttemptContext):
RecordReader[LongWritable, BytesWritable] = {
new FixedLengthBinaryRecordReader
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: the indent is slightly wrong here, should look like

override def createRecordReader(split: InputSplit, context: TaskAttemptContext):
    RecordReader[LongWritable, BytesWritable] = {
  new FixedLengthBinaryRecordReader
}

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
Copy link
Contributor

Choose a reason for hiding this comment

The 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.


}
Loading