Skip to content
Closed
Show file tree
Hide file tree
Changes from 7 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
67 changes: 64 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, StreamFileInputFormat, WholeTextFileInputFormat, ByteInputFormat, 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,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])] = {
Copy link
Contributor

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

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

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

Hey Kevin, is this @note still relevant? using addOnCompleteCallback you might be able to avoid this.

* @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 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]] = {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This has been taken almost directly from
https://github.com/freeman-lab/thunder/blob/master/scala/src/main/scala/thunder/util/Load.scala without the extra formatting to load it as a a list of doubles

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),
Expand Down Expand Up @@ -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
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.FixedLengthBinaryInputFormat

import scala.collection.JavaConversions
import scala.collection.JavaConversions._
import scala.language.implicitConversions
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,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))
Copy link
Contributor

Choose a reason for hiding this comment

The 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
Expand Down
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 {
Copy link
Contributor

Choose a reason for hiding this comment

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

All of these objects should be marked private[spark] so that they don't show up in the public API.


/**
* 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
}
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

}
Loading