From 84035f13028553c595a46a0c0f60b5e448ae3db5 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 30 Jul 2014 13:26:24 +0200 Subject: [PATCH 01/29] adding binary and byte file support spark --- .../scala/org/apache/spark/SparkContext.scala | 22 +++- .../apache/spark/input/BinaryFileInput.scala | 102 ++++++++++++++++++ .../org/apache/spark/rdd/BinaryFileRDD.scala | 48 +++++++++ 3 files changed, 171 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3e6addeaf04a8..af9a1ce935821 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -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.{WholeTextFileInputFormat,ByteInputFormat} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ @@ -510,6 +510,26 @@ 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 byteFiles(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 BinaryFileRDD( + this, + classOf[ByteInputFormat], + classOf[String], + classOf[Array[Byte]], + updateConf, + minPartitions).setName(path) + } + /** * 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), diff --git a/core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala b/core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala new file mode 100644 index 0000000000000..683784c8873b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala @@ -0,0 +1,102 @@ +package org.apache.spark.input + +import scala.collection.JavaConversions._ +import com.google.common.io.{ByteStreams, Closeables} +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader + + +/** + * The new (Hadoop 2.0) InputFormat for while binary files (not be to be confused with the recordreader itself) + */ +@serializable abstract class BinaryFileInputFormat[T] + extends CombineFileInputFormat[String,T] { + override protected def isSplitable(context: JobContext, file: Path): Boolean = false + /** + * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. + */ + def setMaxSplitSize(context: JobContext, minPartitions: Int) { + val files = listStatus(context) + val totalLen = files.map { file => + if (file.isDir) 0L else file.getLen + }.sum + + /** val maxSplitSize = Math.ceil(totalLen * 1.0 / + (if (minPartitions == 0) 1 else minPartitions)).toLong **/ + val maxSplitSize = Math.ceil(totalLen*1.0/files.length).toLong + super.setMaxSplitSize(maxSplitSize) + } + + def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): RecordReader[String,T] + +} + +/** + * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole tiff file + * out in a key-value pair, where the key is the file path and the value is the entire content of + * the file as a TSliceReader (to keep the size information + */ +@serializable abstract class BinaryRecordReader[T]( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends RecordReader[String, T] { + + private val path = split.getPath(index) + private val fs = path.getFileSystem(context.getConfiguration) + + // True means the current file has been processed, then skip it. + private var processed = false + + private val key = path.toString + private var value: T = null.asInstanceOf[T] + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + override def close() = {} + + override def getProgress = if (processed) 1.0f else 0.0f + + override def getCurrentKey = key + + override def getCurrentValue = value + + override def nextKeyValue = { + if (!processed) { + val fileIn = fs.open(path) + val innerBuffer = ByteStreams.toByteArray(fileIn) + value = parseByteArray(innerBuffer) + Closeables.close(fileIn, false) + + processed = true + true + } else { + false + } + } + def parseByteArray(inArray: Array[Byte]): T +} + +/** + * A demo class for extracting just the byte array itself + */ + +@serializable class ByteInputFormat extends BinaryFileInputFormat[Array[Byte]] { + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= + { + new CombineFileRecordReader[String,Array[Byte]](split.asInstanceOf[CombineFileSplit],taContext,classOf[ByteRecordReader]) + } +} + +@serializable class ByteRecordReader( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends BinaryRecordReader[Array[Byte]](split,context,index) { + + def parseByteArray(inArray: Array[Byte]) = inArray +} diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala new file mode 100644 index 0000000000000..7fdc553659f67 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -0,0 +1,48 @@ +package org.apache.spark.rdd + +/** Allows better control of the partitioning + * + */ +import java.text.SimpleDateFormat +import java.util.Date + +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapreduce._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.input.WholeTextFileInputFormat +import org.apache.spark.InterruptibleIterator +import org.apache.spark.Logging +import org.apache.spark.Partition +import org.apache.spark.SerializableWritable +import org.apache.spark.{SparkContext, TaskContext} + +import org.apache.spark.input.BinaryFileInputFormat + +private[spark] class BinaryFileRDD[T]( + sc : SparkContext, + inputFormatClass: Class[_ <: BinaryFileInputFormat[T]], + keyClass: Class[String], + valueClass: Class[T], + @transient conf: Configuration, + minPartitions: Int) + extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { + + override def getPartitions: Array[Partition] = { + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = newJobContext(conf, jobId) + inputFormat.setMaxSplitSize(jobContext, minPartitions) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[Partition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } +} From 1cfa38ae8bb1dcec0416dc9df1ba7a61aef82f9e Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 31 Jul 2014 02:14:57 +0200 Subject: [PATCH 02/29] added apache headers, added datainputstream directly as an output option for more complicated readers (HDF5 perhaps), and renamed several of the functions and files to be more consistent. Also added parallel functions to the java api --- .../scala/org/apache/spark/SparkContext.scala | 28 ++- .../spark/api/java/JavaSparkContext.scala | 64 +++++++ .../apache/spark/input/BinaryFileInput.scala | 102 ----------- .../org/apache/spark/input/RawFileInput.scala | 162 ++++++++++++++++++ .../org/apache/spark/rdd/BinaryFileRDD.scala | 23 ++- 5 files changed, 271 insertions(+), 108 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala create mode 100644 core/src/main/scala/org/apache/spark/input/RawFileInput.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index af9a1ce935821..aa0f8e4ce0101 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -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,ByteInputFormat} +import org.apache.spark.input.{StreamInputFormat, StreamFileInputFormat, WholeTextFileInputFormat, ByteInputFormat} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ @@ -517,11 +517,11 @@ class SparkContext(config: SparkConf) extends Logging { * * @note Small files are preferred, large file is also allowable, but may cause bad performance. */ - def byteFiles(path: String, minPartitions: Int = defaultMinPartitions): RDD[(String, Array[Byte])] = { + 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 BinaryFileRDD( + new RawFileRDD( this, classOf[ByteInputFormat], classOf[String], @@ -530,6 +530,28 @@ class SparkContext(config: SparkConf) extends Logging { minPartitions).setName(path) } + /** + * Get an RDD for a Hadoop-readable dataset as DataInputStreams for each file (useful for binary data) + * Care must be taken to close the files afterwards + * + * @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 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) + } + /** * 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), diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8a5f8088a05ca..7690d009fa2b7 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -20,6 +20,8 @@ package org.apache.spark.api.java import java.util import java.util.{Map => JMap} +import java.io.DataInputStream + import scala.collection.JavaConversions import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -180,6 +182,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 @@ -210,6 +214,66 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def wholeTextFiles(path: String, minPartitions: Int): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(path, minPartitions)) + /** + * 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. + * + *

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 rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + * + *

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)) + + /** + * Read a directory of files as DataInputStreams 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. + * + *

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 rdd = sparkContext.binaryFiles("hdfs://a-hdfs-path")`, + * + *

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)) + /** * 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 diff --git a/core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala b/core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala deleted file mode 100644 index 683784c8873b2..0000000000000 --- a/core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala +++ /dev/null @@ -1,102 +0,0 @@ -package org.apache.spark.input - -import scala.collection.JavaConversions._ -import com.google.common.io.{ByteStreams, Closeables} -import org.apache.hadoop.mapreduce.InputSplit -import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit -import org.apache.hadoop.mapreduce.RecordReader -import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat -import org.apache.hadoop.mapreduce.JobContext -import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader - - -/** - * The new (Hadoop 2.0) InputFormat for while binary files (not be to be confused with the recordreader itself) - */ -@serializable abstract class BinaryFileInputFormat[T] - extends CombineFileInputFormat[String,T] { - override protected def isSplitable(context: JobContext, file: Path): Boolean = false - /** - * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. - */ - def setMaxSplitSize(context: JobContext, minPartitions: Int) { - val files = listStatus(context) - val totalLen = files.map { file => - if (file.isDir) 0L else file.getLen - }.sum - - /** val maxSplitSize = Math.ceil(totalLen * 1.0 / - (if (minPartitions == 0) 1 else minPartitions)).toLong **/ - val maxSplitSize = Math.ceil(totalLen*1.0/files.length).toLong - super.setMaxSplitSize(maxSplitSize) - } - - def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): RecordReader[String,T] - -} - -/** - * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole tiff file - * out in a key-value pair, where the key is the file path and the value is the entire content of - * the file as a TSliceReader (to keep the size information - */ -@serializable abstract class BinaryRecordReader[T]( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends RecordReader[String, T] { - - private val path = split.getPath(index) - private val fs = path.getFileSystem(context.getConfiguration) - - // True means the current file has been processed, then skip it. - private var processed = false - - private val key = path.toString - private var value: T = null.asInstanceOf[T] - override def initialize(split: InputSplit, context: TaskAttemptContext) = {} - override def close() = {} - - override def getProgress = if (processed) 1.0f else 0.0f - - override def getCurrentKey = key - - override def getCurrentValue = value - - override def nextKeyValue = { - if (!processed) { - val fileIn = fs.open(path) - val innerBuffer = ByteStreams.toByteArray(fileIn) - value = parseByteArray(innerBuffer) - Closeables.close(fileIn, false) - - processed = true - true - } else { - false - } - } - def parseByteArray(inArray: Array[Byte]): T -} - -/** - * A demo class for extracting just the byte array itself - */ - -@serializable class ByteInputFormat extends BinaryFileInputFormat[Array[Byte]] { - override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= - { - new CombineFileRecordReader[String,Array[Byte]](split.asInstanceOf[CombineFileSplit],taContext,classOf[ByteRecordReader]) - } -} - -@serializable class ByteRecordReader( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends BinaryRecordReader[Array[Byte]](split,context,index) { - - def parseByteArray(inArray: Array[Byte]) = inArray -} diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala new file mode 100644 index 0000000000000..3f299508fe790 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -0,0 +1,162 @@ +/* + * 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 scala.collection.JavaConversions._ +import com.google.common.io.{ByteStreams, Closeables} +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.fs.{FSDataInputStream, Path} +import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader +import java.io.DataInputStream + + +/** + * A general format for reading whole files in as streams, byte arrays, or other functions to be added + */ +abstract class StreamFileInputFormat[T] + extends CombineFileInputFormat[String,T] { + override protected def isSplitable(context: JobContext, file: Path): Boolean = false + /** + * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. + */ + def setMaxSplitSize(context: JobContext, minPartitions: Int) { + val files = listStatus(context) + val totalLen = files.map { file => + if (file.isDir) 0L else file.getLen + }.sum + + val maxSplitSize = Math.ceil(totalLen*1.0/files.length).toLong + super.setMaxSplitSize(maxSplitSize) + } + + def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): RecordReader[String,T] + +} + +/** + * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] to reading files out as streams + */ +abstract class StreamBasedRecordReader[T]( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends RecordReader[String, T] { + + private val path = split.getPath(index) + private val fs = path.getFileSystem(context.getConfiguration) + + // True means the current file has been processed, then skip it. + private var processed = false + + private val key = path.toString + private var value: T = null.asInstanceOf[T] + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + override def close() = {} + + override def getProgress = if (processed) 1.0f else 0.0f + + override def getCurrentKey = key + + override def getCurrentValue = value + + override def nextKeyValue = { + if (!processed) { + val fileIn: FSDataInputStream = fs.open(path) + value = parseStream(fileIn) + processed = true + true + } else { + false + } + } + + /** + * Parse the stream (and close it afterwards) and return the value as in type T + * @param inStream the stream to be read in + * @return the data formatted as + */ + def parseStream(inStream: DataInputStream): T +} + +/** + * Reads the record in directly as a stream for other objects to manipulate and handle + */ +class StreamRecordReader( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends StreamBasedRecordReader[DataInputStream](split,context,index) { + + def parseStream(inStream: DataInputStream): DataInputStream = inStream +} + +/** + * A class for extracting the information from the file using the BinaryRecordReader (as Byte array) + */ +class StreamInputFormat extends StreamFileInputFormat[DataInputStream] { + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= + { + new CombineFileRecordReader[String,DataInputStream](split.asInstanceOf[CombineFileSplit],taContext,classOf[StreamRecordReader]) + } +} + +/** + * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole binary file + * out in a key-value pair, where the key is the file path and the value is the entire content of + * the file as a byte array + */ +abstract class BinaryRecordReader[T]( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends StreamBasedRecordReader[T](split,context,index) { + + def parseStream(inStream: DataInputStream): T = { + val innerBuffer = ByteStreams.toByteArray(inStream) + Closeables.close(inStream, false) + parseByteArray(innerBuffer) + } + def parseByteArray(inArray: Array[Byte]): T +} + + +class ByteRecordReader( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends BinaryRecordReader[Array[Byte]](split,context,index) { + + def parseByteArray(inArray: Array[Byte]) = inArray +} + +/** + * A class for extracting the information from the file using the BinaryRecordReader (as Byte array) + */ +class ByteInputFormat extends StreamFileInputFormat[Array[Byte]] { + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= + { + new CombineFileRecordReader[String,Array[Byte]](split.asInstanceOf[CombineFileSplit],taContext,classOf[ByteRecordReader]) + } +} + + diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index 7fdc553659f67..e0bacb6dc8db1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd /** Allows better control of the partitioning @@ -18,11 +35,11 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} -import org.apache.spark.input.BinaryFileInputFormat +import org.apache.spark.input.StreamFileInputFormat -private[spark] class BinaryFileRDD[T]( +private[spark] class RawFileRDD[T]( sc : SparkContext, - inputFormatClass: Class[_ <: BinaryFileInputFormat[T]], + inputFormatClass: Class[_ <: StreamFileInputFormat[T]], keyClass: Class[String], valueClass: Class[T], @transient conf: Configuration, From 1622935ca45207206a283642af12d8ee44999fe4 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 31 Jul 2014 02:34:32 +0200 Subject: [PATCH 03/29] changing the line lengths to make jenkins happy --- .../scala/org/apache/spark/SparkContext.scala | 19 ++++++++----- .../spark/api/java/JavaSparkContext.scala | 28 +++++++++++-------- .../org/apache/spark/input/RawFileInput.scala | 24 ++++++++++------ .../org/apache/spark/rdd/BinaryFileRDD.scala | 17 +++-------- 4 files changed, 48 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index aa0f8e4ce0101..4fb226dee9f09 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -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. @@ -511,13 +511,15 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Get an RDD for a Hadoop-readable dataset as byte-streams for each file (useful for binary data) + * 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])] = { + 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 @@ -531,15 +533,18 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Get an RDD for a Hadoop-readable dataset as DataInputStreams for each file (useful for binary data) - * Care must be taken to close the files afterwards + * 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 * @note Small files are preferred, large file is also allowable, but may cause bad performance. */ @DeveloperApi - def dataStreamFiles(path: String, minPartitions: Int = defaultMinPartitions): RDD[(String, DataInputStream)] = { + 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 @@ -1250,7 +1255,7 @@ class SparkContext(config: SparkConf) extends Logging { * If checkSerializable is set, clean will also proactively * check to see if f is serializable and throw a SparkException * if not. - * + * * @param f the closure to clean * @param checkSerializable whether or not to immediately check f for serializability * @throws SparkException if checkSerializable is set but f is not diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 7690d009fa2b7..98c84a779b9da 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -215,9 +215,10 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork new JavaPairRDD(sc.wholeTextFiles(path, minPartitions)) /** - * 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. + * 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. * *

For example, if you have the following files: * {{{ @@ -227,7 +228,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * hdfs://a-hdfs-path/part-nnnnn * }}} * - * Do `JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + * Do + * `JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, * *

then `rdd` contains * {{{ @@ -241,13 +243,14 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * * @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)) + def dataStreamFiles(path: String, minPartitions: Int = defaultMinPartitions): + JavaPairRDD[String,DataInputStream] = new JavaPairRDD(sc.dataStreamFiles(path,minPartitions)) /** - * Read a directory of files as DataInputStreams 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. + * 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. * *

For example, if you have the following files: * {{{ @@ -257,7 +260,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * hdfs://a-hdfs-path/part-nnnnn * }}} * - * Do `JavaPairRDD rdd = sparkContext.binaryFiles("hdfs://a-hdfs-path")`, + * Do + * `JavaPairRDD rdd = sparkContext.binaryFiles("hdfs://a-hdfs-path")`, * *

then `rdd` contains * {{{ @@ -271,8 +275,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * * @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)) + def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): + JavaPairRDD[String,Array[Byte]] = new JavaPairRDD(sc.binaryFiles(path,minPartitions)) /** * Read a directory of text files from HDFS, a local file system (available on all nodes), or any diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index 3f299508fe790..eca82184f587f 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -31,7 +31,8 @@ import java.io.DataInputStream /** - * A general format for reading whole files in as streams, byte arrays, or other functions to be added + * A general format for reading whole files in as streams, byte arrays, + * or other functions to be added */ abstract class StreamFileInputFormat[T] extends CombineFileInputFormat[String,T] { @@ -49,12 +50,14 @@ abstract class StreamFileInputFormat[T] super.setMaxSplitSize(maxSplitSize) } - def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): RecordReader[String,T] + def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): + RecordReader[String,T] } /** - * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] to reading files out as streams + * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] + * to reading files out as streams */ abstract class StreamBasedRecordReader[T]( split: CombineFileSplit, @@ -111,17 +114,20 @@ class StreamRecordReader( } /** - * A class for extracting the information from the file using the BinaryRecordReader (as Byte array) + * A class for extracting the information from the file using the + * BinaryRecordReader (as Byte array) */ class StreamInputFormat extends StreamFileInputFormat[DataInputStream] { override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= { - new CombineFileRecordReader[String,DataInputStream](split.asInstanceOf[CombineFileSplit],taContext,classOf[StreamRecordReader]) + new CombineFileRecordReader[String,DataInputStream]( + split.asInstanceOf[CombineFileSplit],taContext,classOf[StreamRecordReader] + ) } } /** - * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole binary file + * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single binary file * out in a key-value pair, where the key is the file path and the value is the entire content of * the file as a byte array */ @@ -150,12 +156,14 @@ class ByteRecordReader( } /** - * A class for extracting the information from the file using the BinaryRecordReader (as Byte array) + * A class for reading the file using the BinaryRecordReader (as Byte array) */ class ByteInputFormat extends StreamFileInputFormat[Array[Byte]] { override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= { - new CombineFileRecordReader[String,Array[Byte]](split.asInstanceOf[CombineFileSplit],taContext,classOf[ByteRecordReader]) + new CombineFileRecordReader[String,Array[Byte]]( + split.asInstanceOf[CombineFileSplit],taContext,classOf[ByteRecordReader] + ) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index e0bacb6dc8db1..7c31e2b50ab75 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -20,21 +20,10 @@ package org.apache.spark.rdd /** Allows better control of the partitioning * */ -import java.text.SimpleDateFormat -import java.util.Date - import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.input.WholeTextFileInputFormat -import org.apache.spark.InterruptibleIterator -import org.apache.spark.Logging -import org.apache.spark.Partition -import org.apache.spark.SerializableWritable -import org.apache.spark.{SparkContext, TaskContext} - +import org.apache.spark.{Partition, SparkContext} import org.apache.spark.input.StreamFileInputFormat private[spark] class RawFileRDD[T]( @@ -58,7 +47,9 @@ private[spark] class RawFileRDD[T]( val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) { - result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + result(i) = new NewHadoopPartition( + id, i, rawSplits(i).asInstanceOf[InputSplit with Writable] + ) } result } From eacfaa6ca95dd059b943524df9ae037cbce724ed Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 13 Aug 2014 16:06:04 +0200 Subject: [PATCH 04/29] Added FixedLengthBinaryInputFormat and RecordReader from freeman-lab and added them to both the JavaSparkContext and the SparkContext as fixedLengthBinaryFile --- .../scala/org/apache/spark/SparkContext.scala | 38 +++-- .../spark/api/java/JavaSparkContext.scala | 40 +++-- .../input/FixedLengthBinaryInputFormat.scala | 95 ++++++++++++ .../input/FixedLengthBinaryRecordReader.scala | 144 ++++++++++++++++++ 4 files changed, 290 insertions(+), 27 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala create mode 100644 core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4fb226dee9f09..41c0e98ec1d10 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,40 +17,40 @@ package org.apache.spark -import scala.language.implicitConversions - import java.io._ import java.net.URI +import java.util.UUID.randomUUID import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} -import java.util.UUID.randomUUID -import scala.collection.{Map, Set} -import scala.collection.JavaConversions._ -import scala.collection.generic.Growable -import scala.collection.mutable.HashMap -import scala.reflect.{ClassTag, classTag} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, TextInputFormat} -import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} 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.{StreamInputFormat, StreamFileInputFormat, WholeTextFileInputFormat, ByteInputFormat} +import org.apache.spark.input.{ByteInputFormat, FixedLengthBinaryInputFormat, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} +import scala.collection.JavaConversions._ +import scala.collection.generic.Growable +import scala.collection.mutable.HashMap +import scala.collection.{Map, Set} +import scala.language.implicitConversions +import scala.reflect.{ClassTag, classTag} + /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. @@ -557,6 +557,20 @@ class SparkContext(config: SparkConf) extends Logging { 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]] = { + 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), diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 98c84a779b9da..9eab7afed6505 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -17,27 +17,25 @@ package org.apache.spark.api.java +import java.io.DataInputStream import java.util import java.util.{Map => JMap} -import java.io.DataInputStream - -import scala.collection.JavaConversions -import scala.collection.JavaConversions._ -import scala.language.implicitConversions -import scala.reflect.ClassTag - import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration 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._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, RDD} +import scala.collection.JavaConversions +import scala.collection.JavaConversions._ +import scala.language.implicitConversions +import scala.reflect.ClassTag + /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. @@ -214,6 +212,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def wholeTextFiles(path: String, minPartitions: Int): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(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 + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + * @see `wholeTextFiles(path: String, minPartitions: Int)`. + */ + 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 @@ -279,14 +287,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork JavaPairRDD[String,Array[Byte]] = new JavaPairRDD(sc.binaryFiles(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 - * key-value pair, where the key is the path of each file, the value is the content of each file. + * 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) * - * @see `wholeTextFiles(path: String, minPartitions: Int)`. + * @param path Directory to the input data files + * @return An RDD of data with values, JavaRDD[(Array[Byte])] */ - def wholeTextFiles(path: String): JavaPairRDD[String, String] = - new JavaPairRDD(sc.wholeTextFiles(path)) + 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. * diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala new file mode 100644 index 0000000000000..4292a63d7f301 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -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 { + + /** + * 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 + } + } + + /** + * Create a FixedLengthBinaryRecordReader + */ + override def createRecordReader(split: InputSplit, context: TaskAttemptContext): + RecordReader[LongWritable, BytesWritable] = { + new FixedLengthBinaryRecordReader + } + + var recordLength = -1 + +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala new file mode 100644 index 0000000000000..be61617898a7d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -0,0 +1,144 @@ +/* + * 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.lib.input.FileSplit +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} + +/** + * + * 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) + * + */ +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 + +} \ No newline at end of file From f4841dcf89c8c2288ecae9554b9e564ec9fb618d Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 13 Aug 2014 16:14:54 +0200 Subject: [PATCH 05/29] un-optimizing imports, silly intellij --- .../scala/org/apache/spark/SparkContext.scala | 24 +++++++++---------- .../spark/api/java/JavaSparkContext.scala | 19 +++++++++------ .../input/FixedLengthBinaryRecordReader.scala | 2 +- 3 files changed, 25 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 41c0e98ec1d10..049ba326b6921 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,40 +17,40 @@ package org.apache.spark +import scala.language.implicitConversions + import java.io._ import java.net.URI -import java.util.UUID.randomUUID import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} - +import java.util.UUID.randomUUID +import scala.collection.{Map, Set} +import scala.collection.JavaConversions._ +import scala.collection.generic.Growable +import scala.collection.mutable.HashMap +import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, TextInputFormat} -import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} +import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} 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.{ByteInputFormat, FixedLengthBinaryInputFormat, StreamInputFormat, 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._ +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} -import scala.collection.JavaConversions._ -import scala.collection.generic.Growable -import scala.collection.mutable.HashMap -import scala.collection.{Map, Set} -import scala.language.implicitConversions -import scala.reflect.{ClassTag, classTag} - /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 9eab7afed6505..d366befa6240c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -17,25 +17,30 @@ package org.apache.spark.api.java -import java.io.DataInputStream 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 +import scala.reflect.ClassTag + import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} + import org.apache.spark._ +import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, RDD} -import scala.collection.JavaConversions -import scala.collection.JavaConversions._ -import scala.language.implicitConversions -import scala.reflect.ClassTag - /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index be61617898a7d..67f8d49f09aa5 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -22,8 +22,8 @@ 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.lib.input.FileSplit import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.FileSplit /** * From edf58293ee6ff872224afdbe9e580db781caac22 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 13 Aug 2014 16:19:27 +0200 Subject: [PATCH 06/29] fixing line lengths, adding new lines --- .../apache/spark/input/FixedLengthBinaryInputFormat.scala | 8 ++++---- .../spark/input/FixedLengthBinaryRecordReader.scala | 6 ++++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 4292a63d7f301..a59317afc0e0f 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -23,9 +23,9 @@ 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. + * 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 { @@ -92,4 +92,4 @@ class FixedLengthBinaryInputFormat extends FileInputFormat[LongWritable, BytesWr var recordLength = -1 -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index 67f8d49f09aa5..e7ea91809aac5 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -56,7 +56,9 @@ class FixedLengthBinaryRecordReader extends RecordReader[LongWritable, BytesWrit 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 + case _ => Math.min( + ((currentPosition - splitStart) / (splitEnd - splitStart)).toFloat, 1.0 + ).toFloat } } @@ -141,4 +143,4 @@ class FixedLengthBinaryRecordReader extends RecordReader[LongWritable, BytesWrit var recordKey: LongWritable = null var recordValue: BytesWritable = null -} \ No newline at end of file +} From 9a313d571fc781cf440466dc83b881ff8cb11cad Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 14 Aug 2014 20:04:51 +0200 Subject: [PATCH 07/29] making classes that needn't be public private, adding automatic file closure, adding new tests --- .../scala/org/apache/spark/SparkContext.scala | 10 ++-- .../spark/api/java/JavaSparkContext.scala | 6 +-- .../input/FixedLengthBinaryInputFormat.scala | 4 +- .../input/FixedLengthBinaryRecordReader.scala | 2 +- .../org/apache/spark/input/RawFileInput.scala | 23 +++++--- .../org/apache/spark/rdd/BinaryFileRDD.scala | 5 +- .../java/org/apache/spark/JavaAPISuite.java | 22 ++++++++ .../scala/org/apache/spark/FileSuite.scala | 54 +++++++++++++++++++ 8 files changed, 107 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 049ba326b6921..d12dd8a894bb2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -523,7 +523,7 @@ class SparkContext(config: SparkConf) extends Logging { val job = new NewHadoopJob(hadoopConfiguration) NewFileInputFormat.addInputPath(job, new Path(path)) val updateConf = job.getConfiguration - new RawFileRDD( + new BinaryFileRDD( this, classOf[ByteInputFormat], classOf[String], @@ -548,7 +548,7 @@ class SparkContext(config: SparkConf) extends Logging { val job = new NewHadoopJob(hadoopConfiguration) NewFileInputFormat.addInputPath(job, new Path(path)) val updateConf = job.getConfiguration - new RawFileRDD( + new BinaryFileRDD( this, classOf[StreamInputFormat], classOf[String], @@ -565,9 +565,9 @@ class SparkContext(config: SparkConf) extends Logging { * @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]] = { - val lines = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path) - val data = lines.map{ case (k, v) => v.getBytes} + def binaryRecords(path: String): RDD[Array[Byte]] = { + val br = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path) + val data = br.map{ case (k, v) => v.getBytes} data } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index d366befa6240c..a407263aa8dd0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -289,7 +289,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * @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)) + 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 @@ -299,8 +299,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * @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)) + def binaryRecords(path: String): JavaRDD[Array[Byte]] = { + new JavaRDD(sc.binaryRecords(path)) } /** Get an RDD for a Hadoop SequenceFile with given key and value types. diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index a59317afc0e0f..309fd578b532f 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAt * a parameter recordLength in the Hadoop configuration. */ -object FixedLengthBinaryInputFormat { +private[spark] object FixedLengthBinaryInputFormat { /** * This function retrieves the recordLength by checking the configuration parameter @@ -42,7 +42,7 @@ object FixedLengthBinaryInputFormat { } -class FixedLengthBinaryInputFormat extends FileInputFormat[LongWritable, BytesWritable] { +private[spark] class FixedLengthBinaryInputFormat extends FileInputFormat[LongWritable, BytesWritable] { /** diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index e7ea91809aac5..a292a1e41d912 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit * VALUE = the record itself (BytesWritable) * */ -class FixedLengthBinaryRecordReader extends RecordReader[LongWritable, BytesWritable] { +private[spark] class FixedLengthBinaryRecordReader extends RecordReader[LongWritable, BytesWritable] { override def close() { if (fileInputStream != null) { diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index eca82184f587f..fa79ed7af3513 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -73,8 +73,18 @@ abstract class StreamBasedRecordReader[T]( private val key = path.toString private var value: T = null.asInstanceOf[T] + // the file to be read when nextkeyvalue is called + private lazy val fileIn: FSDataInputStream = fs.open(path) + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} - override def close() = {} + override def close() = { + // make sure the file is closed + try { + fileIn.close() + } catch { + case ioe: java.io.IOException => // do nothing + } + } override def getProgress = if (processed) 1.0f else 0.0f @@ -82,9 +92,10 @@ abstract class StreamBasedRecordReader[T]( override def getCurrentValue = value + override def nextKeyValue = { if (!processed) { - val fileIn: FSDataInputStream = fs.open(path) + value = parseStream(fileIn) processed = true true @@ -104,7 +115,7 @@ abstract class StreamBasedRecordReader[T]( /** * Reads the record in directly as a stream for other objects to manipulate and handle */ -class StreamRecordReader( +private[spark] class StreamRecordReader( split: CombineFileSplit, context: TaskAttemptContext, index: Integer) @@ -117,7 +128,7 @@ class StreamRecordReader( * A class for extracting the information from the file using the * BinaryRecordReader (as Byte array) */ -class StreamInputFormat extends StreamFileInputFormat[DataInputStream] { +private[spark] class StreamInputFormat extends StreamFileInputFormat[DataInputStream] { override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= { new CombineFileRecordReader[String,DataInputStream]( @@ -146,7 +157,7 @@ abstract class BinaryRecordReader[T]( } -class ByteRecordReader( +private[spark] class ByteRecordReader( split: CombineFileSplit, context: TaskAttemptContext, index: Integer) @@ -158,7 +169,7 @@ class ByteRecordReader( /** * A class for reading the file using the BinaryRecordReader (as Byte array) */ -class ByteInputFormat extends StreamFileInputFormat[Array[Byte]] { +private[spark] class ByteInputFormat extends StreamFileInputFormat[Array[Byte]] { override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= { new CombineFileRecordReader[String,Array[Byte]]( diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index 7c31e2b50ab75..c7dc50820d59b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -23,10 +23,10 @@ package org.apache.spark.rdd import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{Partition, SparkContext} +import org.apache.spark.{InterruptibleIterator, TaskContext, Partition, SparkContext} import org.apache.spark.input.StreamFileInputFormat -private[spark] class RawFileRDD[T]( +private[spark] class BinaryFileRDD[T]( sc : SparkContext, inputFormatClass: Class[_ <: StreamFileInputFormat[T]], keyClass: Class[String], @@ -35,6 +35,7 @@ private[spark] class RawFileRDD[T]( minPartitions: Int) extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { + override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance inputFormat match { diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e8bd65f8e4507..5bf2dbccf28f5 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -836,6 +836,28 @@ public Tuple2 call(Tuple2 pair) { Assert.assertEquals(pairs, readRDD.collect()); } + @Test + public void binaryFiles() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); + + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + Files.write(content1, file1); + File file2 = new File(tempDirName + "/part-00001"); + Files.write(content2, file2); + + JavaPairRDD readRDD = sc.binaryFiles(tempDirName,3); + List> result = readRDD.collect(); + for (Tuple2 res : result) { + if (res._1()==file1.toString()) + Assert.assertArrayEquals(content1,res._2()); + else + Assert.assertArrayEquals(content2,res._2()); + } + } + @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index c70e22cf09433..37db288430692 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -224,6 +224,60 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } + test("byte stream input") { + sc = new SparkContext("local", "test") + val outputDir = new File(tempDir, "output").getAbsolutePath + val outFile = new File(outputDir, "part-00000.bin") + val outFileName = outFile.toPath().toString() + + // create file + val testOutput = Array[Byte](1,2,3,4,5,6) + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName) + val (infile: String, indata: Array[Byte]) = inRdd.first + + // Try reading the output back as an object file + assert(infile === outFileName) + assert(indata === testOutput) + } + + test("fixed length byte stream input") { + // a fixed length of 6 bytes + + sc = new SparkContext("local", "test") + + val outputDir = new File(tempDir, "output").getAbsolutePath + val outFile = new File(outputDir, "part-00000.bin") + val outFileName = outFile.toPath().toString() + + // create file + val testOutput = Array[Byte](1,2,3,4,5,6) + val testOutputCopies = 10 + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + for(i <- 1 to testOutputCopies) channel.write(bbuf) + channel.close() + file.close() + sc.hadoopConfiguration.setInt("recordLength",testOutput.length) + + val inRdd = sc.binaryRecords(outFileName) + // make sure there are enough elements + assert(inRdd.count== testOutputCopies) + + // now just compare the first one + val indata: Array[Byte] = inRdd.first + assert(indata === testOutput) + } + test("file caching") { sc = new SparkContext("local", "test") val out = new FileWriter(tempDir + "/input") From df8e528e72fda1bd12ecb52b1d836c3a70a1d37f Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 14 Aug 2014 20:16:43 +0200 Subject: [PATCH 08/29] fixed line lengths and changed java test --- .../input/FixedLengthBinaryInputFormat.scala | 3 ++- .../input/FixedLengthBinaryRecordReader.scala | 3 ++- .../java/org/apache/spark/JavaAPISuite.java | 19 +++++++++++-------- 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 309fd578b532f..87993e1ecb0cf 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -42,7 +42,8 @@ private[spark] object FixedLengthBinaryInputFormat { } -private[spark] class FixedLengthBinaryInputFormat extends FileInputFormat[LongWritable, BytesWritable] { +private[spark] class FixedLengthBinaryInputFormat + extends FileInputFormat[LongWritable, BytesWritable] { /** diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index a292a1e41d912..eb27a98fe09f2 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -37,7 +37,8 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit * VALUE = the record itself (BytesWritable) * */ -private[spark] class FixedLengthBinaryRecordReader extends RecordReader[LongWritable, BytesWritable] { +private[spark] class FixedLengthBinaryRecordReader + extends RecordReader[LongWritable, BytesWritable] { override def close() { if (fileInputStream != null) { diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 5bf2dbccf28f5..96865d7b736b1 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,6 +18,8 @@ package org.apache.spark; import java.io.*; +import java.nio.channels.FileChannel; +import java.nio.ByteBuffer; import java.net.URI; import java.util.*; @@ -840,21 +842,22 @@ public Tuple2 call(Tuple2 pair) { public void binaryFiles() throws Exception { // Reusing the wholeText files example byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); - byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); + String tempDirName = tempDir.getAbsolutePath(); File file1 = new File(tempDirName + "/part-00000"); - Files.write(content1, file1); - File file2 = new File(tempDirName + "/part-00001"); - Files.write(content2, file2); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + channel1.write(bbuf); + JavaPairRDD readRDD = sc.binaryFiles(tempDirName,3); List> result = readRDD.collect(); for (Tuple2 res : result) { - if (res._1()==file1.toString()) - Assert.assertArrayEquals(content1,res._2()); - else - Assert.assertArrayEquals(content2,res._2()); + Assert.assertArrayEquals(content1, res._2()); } } From bc5c0b96a8782402c9536cc4879417494ed3f37b Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 14 Aug 2014 20:33:56 +0200 Subject: [PATCH 09/29] made minor stylistic adjustments from mateiz --- .../spark/input/FixedLengthBinaryInputFormat.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 87993e1ecb0cf..646fe23738a66 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -77,9 +77,10 @@ private[spark] class FixedLengthBinaryInputFormat // 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 + if (defaultSize < recordLength) { + recordLength.toLong + } else { + (Math.floor(defaultSize / recordLength) * recordLength).toLong } } @@ -87,8 +88,8 @@ private[spark] class FixedLengthBinaryInputFormat * Create a FixedLengthBinaryRecordReader */ override def createRecordReader(split: InputSplit, context: TaskAttemptContext): - RecordReader[LongWritable, BytesWritable] = { - new FixedLengthBinaryRecordReader + RecordReader[LongWritable, BytesWritable] = { + new FixedLengthBinaryRecordReader } var recordLength = -1 From f032bc0e0d62474161fb9fb734e898871d3fdcd2 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 14 Aug 2014 22:58:09 +0200 Subject: [PATCH 10/29] fixed bug in path name, renamed tests --- core/src/test/scala/org/apache/spark/FileSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 37db288430692..b723f79c7fb3f 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -224,10 +224,10 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } - test("byte stream input") { + test("binary file input as byte array") { sc = new SparkContext("local", "test") - val outputDir = new File(tempDir, "output").getAbsolutePath - val outFile = new File(outputDir, "part-00000.bin") + val outputDir = new File(tempDir).getAbsolutePath + val outFile = new File(outputDir, "record-bytestream-00000.bin") val outFileName = outFile.toPath().toString() // create file @@ -248,13 +248,13 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(indata === testOutput) } - test("fixed length byte stream input") { + test("fixed record length binary file as byte array") { // a fixed length of 6 bytes sc = new SparkContext("local", "test") - val outputDir = new File(tempDir, "output").getAbsolutePath - val outFile = new File(outputDir, "part-00000.bin") + val outputDir = new File(tempDir).getAbsolutePath + val outFile = new File(outputDir, "record-bytestream-00000.bin") val outFileName = outFile.toPath().toString() // create file From 5deb79eebe9ee22878bbae4018ed51731e9838f6 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Sun, 7 Sep 2014 13:34:36 +0200 Subject: [PATCH 11/29] added new portabledatastream to code so that it can be serialized correctly --- .../scala/org/apache/spark/SparkContext.scala | 8 +- .../spark/api/java/JavaSparkContext.scala | 4 +- .../org/apache/spark/input/RawFileInput.scala | 99 ++++++++++++------- .../scala/org/apache/spark/FileSuite.scala | 11 +-- pom.xml | 5 + 5 files changed, 80 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d12dd8a894bb2..1b6ad249bc5c9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -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.{StreamInputFormat, StreamFileInputFormat, WholeTextFileInputFormat, ByteInputFormat, FixedLengthBinaryInputFormat} +import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, ByteInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ @@ -533,7 +533,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Get an RDD for a Hadoop-readable dataset as DataInputStreams for each file + * Get an RDD for a Hadoop-readable dataset as PortableDataStream for each file * (useful for binary data) * * @@ -544,7 +544,7 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def dataStreamFiles(path: String, minPartitions: Int = defaultMinPartitions): - RDD[(String, DataInputStream)] = { + RDD[(String, PortableDataStream)] = { val job = new NewHadoopJob(hadoopConfiguration) NewFileInputFormat.addInputPath(job, new Path(path)) val updateConf = job.getConfiguration @@ -552,7 +552,7 @@ class SparkContext(config: SparkConf) extends Logging { this, classOf[StreamInputFormat], classOf[String], - classOf[DataInputStream], + classOf[PortableDataStream], updateConf, minPartitions).setName(path) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index a407263aa8dd0..dcaaccb437168 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -23,7 +23,7 @@ import java.util.{Map => JMap} import java.io.DataInputStream import org.apache.hadoop.io.{BytesWritable, LongWritable} -import org.apache.spark.input.FixedLengthBinaryInputFormat +import org.apache.spark.input.{PortableDataStream, FixedLengthBinaryInputFormat} import scala.collection.JavaConversions import scala.collection.JavaConversions._ @@ -257,7 +257,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * @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)) + JavaPairRDD[String,PortableDataStream] = new JavaPairRDD(sc.dataStreamFiles(path,minPartitions)) /** * Read a directory of files as DataInputStream from HDFS, diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index fa79ed7af3513..4a8e543adecfa 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -51,40 +51,66 @@ abstract class StreamFileInputFormat[T] } def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): - RecordReader[String,T] + RecordReader[String,T] } +/** + * A class that allows DataStreams to be serialized and moved around by not creating them + * until they need to be read + * @param split + * @param context + * @param index + */ +class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, index: Integer) + extends Serializable { + private var path = "" + private var fileIn: FSDataInputStream = null.asInstanceOf[FSDataInputStream] + private var isOpen = false + + def open(): FSDataInputStream= { + val pathp = split.getPath(index) + path = pathp.toString + val fs = pathp.getFileSystem(context.getConfiguration) + fileIn = fs.open(pathp) + isOpen=true + fileIn + } + + def close() = { + if (isOpen) { + try { + fileIn.close() + isOpen=false + } catch { + case ioe: java.io.IOException => // do nothing + } + } + } + def getPath(): String = path +} + /** * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] * to reading files out as streams */ abstract class StreamBasedRecordReader[T]( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) extends RecordReader[String, T] { - private val path = split.getPath(index) - private val fs = path.getFileSystem(context.getConfiguration) + // True means the current file has been processed, then skip it. private var processed = false - private val key = path.toString + private var key = "" private var value: T = null.asInstanceOf[T] - // the file to be read when nextkeyvalue is called - private lazy val fileIn: FSDataInputStream = fs.open(path) + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} - override def close() = { - // make sure the file is closed - try { - fileIn.close() - } catch { - case ioe: java.io.IOException => // do nothing - } - } + override def close() = {} override def getProgress = if (processed) 1.0f else 0.0f @@ -93,10 +119,13 @@ abstract class StreamBasedRecordReader[T]( override def getCurrentValue = value + override def nextKeyValue = { if (!processed) { - + val fileIn = new PortableDataStream(split,context,index) + key = fileIn.getPath value = parseStream(fileIn) + fileIn.close() // if it has not been open yet, close does nothing processed = true true } else { @@ -109,29 +138,29 @@ abstract class StreamBasedRecordReader[T]( * @param inStream the stream to be read in * @return the data formatted as */ - def parseStream(inStream: DataInputStream): T + def parseStream(inStream: PortableDataStream): T } /** * Reads the record in directly as a stream for other objects to manipulate and handle */ private[spark] class StreamRecordReader( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends StreamBasedRecordReader[DataInputStream](split,context,index) { + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends StreamBasedRecordReader[PortableDataStream](split,context,index) { - def parseStream(inStream: DataInputStream): DataInputStream = inStream + def parseStream(inStream: PortableDataStream): PortableDataStream = inStream } /** * A class for extracting the information from the file using the * BinaryRecordReader (as Byte array) */ -private[spark] class StreamInputFormat extends StreamFileInputFormat[DataInputStream] { +private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= { - new CombineFileRecordReader[String,DataInputStream]( + new CombineFileRecordReader[String,PortableDataStream]( split.asInstanceOf[CombineFileSplit],taContext,classOf[StreamRecordReader] ) } @@ -143,12 +172,13 @@ private[spark] class StreamInputFormat extends StreamFileInputFormat[DataInputSt * the file as a byte array */ abstract class BinaryRecordReader[T]( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) extends StreamBasedRecordReader[T](split,context,index) { - def parseStream(inStream: DataInputStream): T = { + def parseStream(inpStream: PortableDataStream): T = { + val inStream = inpStream.open() val innerBuffer = ByteStreams.toByteArray(inStream) Closeables.close(inStream, false) parseByteArray(innerBuffer) @@ -157,13 +187,14 @@ abstract class BinaryRecordReader[T]( } + private[spark] class ByteRecordReader( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) extends BinaryRecordReader[Array[Byte]](split,context,index) { - def parseByteArray(inArray: Array[Byte]) = inArray + override def parseByteArray(inArray: Array[Byte]) = inArray } /** diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index b723f79c7fb3f..2bb6d809cc174 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -226,9 +226,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test("binary file input as byte array") { sc = new SparkContext("local", "test") - val outputDir = new File(tempDir).getAbsolutePath - val outFile = new File(outputDir, "record-bytestream-00000.bin") - val outFileName = outFile.toPath().toString() + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() // create file val testOutput = Array[Byte](1,2,3,4,5,6) @@ -252,10 +251,8 @@ class FileSuite extends FunSuite with LocalSparkContext { // a fixed length of 6 bytes sc = new SparkContext("local", "test") - - val outputDir = new File(tempDir).getAbsolutePath - val outFile = new File(outputDir, "record-bytestream-00000.bin") - val outFileName = outFile.toPath().toString() + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() // create file val testOutput = Array[Byte](1,2,3,4,5,6) diff --git a/pom.xml b/pom.xml index ae97bf03c53a2..cfd59cf463131 100644 --- a/pom.xml +++ b/pom.xml @@ -90,6 +90,7 @@ bagel graphx mllib + imglib tools streaming sql/catalyst @@ -743,6 +744,10 @@ jackson-mapper-asl 1.8.8 + + io.scif + pom-scifio + From 12e7be1ad035667428d821d285b7458d425e34e9 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Sun, 7 Sep 2014 13:36:45 +0200 Subject: [PATCH 12/29] removing imglib from maven (definitely not ready yet) --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index cfd59cf463131..6b8e301107573 100644 --- a/pom.xml +++ b/pom.xml @@ -90,7 +90,6 @@ bagel graphx mllib - imglib tools streaming sql/catalyst From 441f79a0e435964efc1b5f5a2a1461275a4dc439 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Sun, 7 Sep 2014 13:59:56 +0200 Subject: [PATCH 13/29] fixed a few small comments and dependency --- .../scala/org/apache/spark/input/RawFileInput.scala | 11 +++++++---- pom.xml | 4 ---- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index 4a8e543adecfa..377df6e2cda38 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -58,9 +58,6 @@ abstract class StreamFileInputFormat[T] /** * A class that allows DataStreams to be serialized and moved around by not creating them * until they need to be read - * @param split - * @param context - * @param index */ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, index: Integer) extends Serializable { @@ -68,7 +65,10 @@ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, i private var fileIn: FSDataInputStream = null.asInstanceOf[FSDataInputStream] private var isOpen = false - def open(): FSDataInputStream= { + /** + * create a new DataInputStream from the split and context + */ + def open(): FSDataInputStream = { val pathp = split.getPath(index) path = pathp.toString val fs = pathp.getFileSystem(context.getConfiguration) @@ -77,6 +77,9 @@ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, i fileIn } + /** + * close the file (if it is already open) + */ def close() = { if (isOpen) { try { diff --git a/pom.xml b/pom.xml index 6b8e301107573..ae97bf03c53a2 100644 --- a/pom.xml +++ b/pom.xml @@ -743,10 +743,6 @@ jackson-mapper-asl 1.8.8 - - io.scif - pom-scifio - From a01c9cf932e45dc5e13401d41d9ad3f2eb55e7a6 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Tue, 16 Sep 2014 18:12:30 +0200 Subject: [PATCH 14/29] Update RawFileInput.scala trying to fix bug where name appears blank --- .../scala/org/apache/spark/input/RawFileInput.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index 377df6e2cda38..ce38176a66fdc 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -61,16 +61,22 @@ abstract class StreamFileInputFormat[T] */ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, index: Integer) extends Serializable { - private var path = "" + private var fileIn: FSDataInputStream = null.asInstanceOf[FSDataInputStream] private var isOpen = false + /** + * Calculate the path name independently of opening the file + */ + private lazy val path = { + val pathp = split.getPath(index) + path = pathp.toString + } /** * create a new DataInputStream from the split and context */ def open(): FSDataInputStream = { val pathp = split.getPath(index) - path = pathp.toString val fs = pathp.getFileSystem(context.getConfiguration) fileIn = fs.open(pathp) isOpen=true @@ -126,9 +132,9 @@ abstract class StreamBasedRecordReader[T]( override def nextKeyValue = { if (!processed) { val fileIn = new PortableDataStream(split,context,index) - key = fileIn.getPath value = parseStream(fileIn) fileIn.close() // if it has not been open yet, close does nothing + key = fileIn.getPath processed = true true } else { From 932a2066edc45c666d94253b2c3e5f7763638b81 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Tue, 16 Sep 2014 18:15:14 +0200 Subject: [PATCH 15/29] Update RawFileInput.scala typo in path value --- core/src/main/scala/org/apache/spark/input/RawFileInput.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index ce38176a66fdc..d820e0efcea4a 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -69,7 +69,7 @@ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, i */ private lazy val path = { val pathp = split.getPath(index) - path = pathp.toString + pathp.toString } /** From 238c83cc9eeab7012aad1a3e2660aae31073a56d Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 1 Oct 2014 11:36:09 +0200 Subject: [PATCH 16/29] fixed several scala-style issues, changed structure of binaryFiles, removed excessive classes added new tests. The caching tests still have a serialization issue, but that should be easily fixed as well. --- .../scala/org/apache/spark/SparkContext.scala | 36 ++++------ .../spark/api/java/JavaSparkContext.scala | 14 ++-- .../org/apache/spark/input/RawFileInput.scala | 59 +++++++---------- .../java/org/apache/spark/JavaAPISuite.java | 65 +++++++++++++++++-- .../scala/org/apache/spark/FileSuite.scala | 50 ++++++++++++-- 5 files changed, 146 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1b6ad249bc5c9..80490fecd978b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -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.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, ByteInputFormat, FixedLengthBinaryInputFormat} +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._ @@ -510,27 +510,6 @@ 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 BinaryFileRDD( - this, - classOf[ByteInputFormat], - classOf[String], - classOf[Array[Byte]], - updateConf, - minPartitions).setName(path) - } /** * Get an RDD for a Hadoop-readable dataset as PortableDataStream for each file @@ -543,7 +522,7 @@ class SparkContext(config: SparkConf) extends Logging { * @note Small files are preferred, large file is also allowable, but may cause bad performance. */ @DeveloperApi - def dataStreamFiles(path: String, minPartitions: Int = defaultMinPartitions): + def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): RDD[(String, PortableDataStream)] = { val job = new NewHadoopJob(hadoopConfiguration) NewFileInputFormat.addInputPath(job, new Path(path)) @@ -563,10 +542,17 @@ class SparkContext(config: SparkConf) extends Logging { * 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): RDD[Array[Byte]] = { - val br = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path) + 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} data } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index dcaaccb437168..8c869f7f70f65 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -36,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} @@ -256,8 +256,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * * @param minPartitions A suggestion value of the minimal splitting number for input data. */ - def dataStreamFiles(path: String, minPartitions: Int = defaultMinPartitions): - JavaPairRDD[String,PortableDataStream] = new JavaPairRDD(sc.dataStreamFiles(path,minPartitions)) + def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): + JavaPairRDD[String,PortableDataStream] = new JavaPairRDD(sc.binaryFiles(path,minPartitions)) /** * Read a directory of files as DataInputStream from HDFS, @@ -288,8 +288,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * * @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)) + def binaryArrays(path: String, minPartitions: Int = defaultMinPartitions): + JavaPairRDD[String, Array[Byte]] = new JavaPairRDD(sc.binaryFiles(path,minPartitions).mapValues(_.toArray())) /** * Load data from a flat binary file, assuming each record is a set of numbers @@ -299,8 +299,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * @param path Directory to the input data files * @return An RDD of data with values, JavaRDD[(Array[Byte])] */ - def binaryRecords(path: String): JavaRDD[Array[Byte]] = { - new JavaRDD(sc.binaryRecords(path)) + 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. diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index d820e0efcea4a..0ff5127e5df0e 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -46,7 +46,7 @@ abstract class StreamFileInputFormat[T] if (file.isDir) 0L else file.getLen }.sum - val maxSplitSize = Math.ceil(totalLen*1.0/files.length).toLong + val maxSplitSize = Math.ceil(totalLen * 1.0 / files.length).toLong super.setMaxSplitSize(maxSplitSize) } @@ -61,8 +61,10 @@ abstract class StreamFileInputFormat[T] */ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, index: Integer) extends Serializable { - + // transient forces file to be reopened after being moved (serialization) + @transient private var fileIn: FSDataInputStream = null.asInstanceOf[FSDataInputStream] + @transient private var isOpen = false /** * Calculate the path name independently of opening the file @@ -76,13 +78,25 @@ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, i * create a new DataInputStream from the split and context */ def open(): FSDataInputStream = { - val pathp = split.getPath(index) - val fs = pathp.getFileSystem(context.getConfiguration) - fileIn = fs.open(pathp) - isOpen=true + if (!isOpen) { + val pathp = split.getPath(index) + val fs = pathp.getFileSystem(context.getConfiguration) + fileIn = fs.open(pathp) + isOpen=true + } fileIn } + /** + * Read the file as a byte array + */ + def toArray(): Array[Byte] = { + open() + val innerBuffer = ByteStreams.toByteArray(fileIn) + close() + innerBuffer + } + /** * close the file (if it is already open) */ @@ -131,7 +145,7 @@ abstract class StreamBasedRecordReader[T]( override def nextKeyValue = { if (!processed) { - val fileIn = new PortableDataStream(split,context,index) + val fileIn = new PortableDataStream(split, context, index) value = parseStream(fileIn) fileIn.close() // if it has not been open yet, close does nothing key = fileIn.getPath @@ -157,7 +171,7 @@ private[spark] class StreamRecordReader( split: CombineFileSplit, context: TaskAttemptContext, index: Integer) - extends StreamBasedRecordReader[PortableDataStream](split,context,index) { + extends StreamBasedRecordReader[PortableDataStream](split, context, index) { def parseStream(inStream: PortableDataStream): PortableDataStream = inStream } @@ -170,7 +184,7 @@ private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDat override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= { new CombineFileRecordReader[String,PortableDataStream]( - split.asInstanceOf[CombineFileSplit],taContext,classOf[StreamRecordReader] + split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader] ) } } @@ -193,29 +207,4 @@ abstract class BinaryRecordReader[T]( parseByteArray(innerBuffer) } def parseByteArray(inArray: Array[Byte]): T -} - - - -private[spark] class ByteRecordReader( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends BinaryRecordReader[Array[Byte]](split,context,index) { - - override def parseByteArray(inArray: Array[Byte]) = inArray -} - -/** - * A class for reading the file using the BinaryRecordReader (as Byte array) - */ -private[spark] class ByteInputFormat extends StreamFileInputFormat[Array[Byte]] { - override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= - { - new CombineFileRecordReader[String,Array[Byte]]( - split.asInstanceOf[CombineFileSplit],taContext,classOf[ByteRecordReader] - ) - } -} - - +} \ No newline at end of file diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 96865d7b736b1..b4dc8de323c1d 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -23,6 +23,7 @@ import java.net.URI; import java.util.*; +import org.apache.spark.input.PortableDataStream; import scala.Tuple2; import scala.Tuple3; import scala.Tuple4; @@ -852,12 +853,68 @@ public void binaryFiles() throws Exception { FileChannel channel1 = fos1.getChannel(); ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); channel1.write(bbuf); + channel1.close(); + JavaPairRDD readRDD = sc.binaryFiles(tempDirName,3); + List> result = readRDD.collect(); + for (Tuple2 res : result) { + Assert.assertArrayEquals(content1, res._2().toArray()); + } + } + + @Test + public void binaryFilesCaching() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + + + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + channel1.write(bbuf); + channel1.close(); + + JavaPairRDD readRDD = sc.binaryFiles(tempDirName,3).cache(); + readRDD.foreach(new VoidFunction>() { + @Override + public void call(Tuple2 stringPortableDataStreamTuple2) throws Exception { + stringPortableDataStreamTuple2._2().getPath(); + stringPortableDataStreamTuple2._2().toArray(); // force the file to read + } + }); + + List> result = readRDD.collect(); + for (Tuple2 res : result) { + Assert.assertArrayEquals(content1, res._2().toArray()); + } + } + @Test + public void binaryRecords() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark isn't always easy to use.\n".getBytes("utf-8"); + int numOfCopies = 10; + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + + for (int i=0;i readRDD = sc.binaryFiles(tempDirName,3); - List> result = readRDD.collect(); - for (Tuple2 res : result) { - Assert.assertArrayEquals(content1, res._2()); + JavaRDD readRDD = sc.binaryRecords(tempDirName,content1.length); + Assert.assertEquals(numOfCopies,readRDD.count()); + List result = readRDD.collect(); + for (byte[] res : result) { + Assert.assertArrayEquals(content1, res); } } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 2bb6d809cc174..d2913c7d2d479 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark import java.io.{File, FileWriter} +import org.apache.spark.input.PortableDataStream + import scala.io.Source import com.google.common.io.Files @@ -240,35 +242,69 @@ class FileSuite extends FunSuite with LocalSparkContext { file.close() val inRdd = sc.binaryFiles(outFileName) - val (infile: String, indata: Array[Byte]) = inRdd.first + val (infile: String, indata: PortableDataStream) = inRdd.first // Try reading the output back as an object file assert(infile === outFileName) - assert(indata === testOutput) + assert(indata.toArray === testOutput) + } + + test("portabledatastream caching tests") { + sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1,2,3,4,5,6) + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName).cache() + inRdd.foreach{ + curData: (String, PortableDataStream) => + curData._2.toArray() // force the file to read + } + val mappedRdd = inRdd.map{ + curData: (String, PortableDataStream) => + (curData._2.getPath(),curData._2) + } + val (infile: String, indata: PortableDataStream) = mappedRdd.first + + // Try reading the output back as an object file + + assert(indata.toArray === testOutput) } test("fixed record length binary file as byte array") { // a fixed length of 6 bytes sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") val outFileName = outFile.getAbsolutePath() // create file val testOutput = Array[Byte](1,2,3,4,5,6) val testOutputCopies = 10 - val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file val file = new java.io.FileOutputStream(outFile) val channel = file.getChannel - for(i <- 1 to testOutputCopies) channel.write(bbuf) + for(i <- 1 to testOutputCopies) { + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + channel.write(bbuf) + } channel.close() file.close() - sc.hadoopConfiguration.setInt("recordLength",testOutput.length) - val inRdd = sc.binaryRecords(outFileName) + val inRdd = sc.binaryRecords(outFileName, testOutput.length) // make sure there are enough elements - assert(inRdd.count== testOutputCopies) + assert(inRdd.count == testOutputCopies) // now just compare the first one val indata: Array[Byte] = inRdd.first From 19812a83df8a4852412feb7dec7f42126b0b139e Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 1 Oct 2014 13:35:34 +0200 Subject: [PATCH 17/29] Fixed the serialization issue with PortableDataStream since neither CombineFileSplit nor TaskAttemptContext implement the Serializable interface, by using ByteArrays for storing both and then recreating the objects from these bytearrays as needed. --- .../spark/api/java/JavaSparkContext.scala | 3 +- .../org/apache/spark/input/RawFileInput.scala | 44 ++++++++++++++++--- .../java/org/apache/spark/JavaAPISuite.java | 1 - .../scala/org/apache/spark/FileSuite.scala | 37 ++++++++++++++++ 4 files changed, 78 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8c869f7f70f65..ef107b27c87fb 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -289,7 +289,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * @param minPartitions A suggestion value of the minimal splitting number for input data. */ def binaryArrays(path: String, minPartitions: Int = defaultMinPartitions): - JavaPairRDD[String, Array[Byte]] = new JavaPairRDD(sc.binaryFiles(path,minPartitions).mapValues(_.toArray())) + JavaPairRDD[String, Array[Byte]] = + new JavaPairRDD(sc.binaryFiles(path,minPartitions).mapValues(_.toArray())) /** * Load data from a flat binary file, assuming each record is a set of numbers diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index 0ff5127e5df0e..1f3989995373c 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -20,6 +20,7 @@ package org.apache.spark.input import scala.collection.JavaConversions._ import com.google.common.io.{ByteStreams, Closeables} import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext @@ -27,7 +28,7 @@ import org.apache.hadoop.fs.{FSDataInputStream, Path} import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader -import java.io.DataInputStream +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutputStream, DataInputStream} /** @@ -58,17 +59,50 @@ abstract class StreamFileInputFormat[T] /** * A class that allows DataStreams to be serialized and moved around by not creating them * until they need to be read + * @note TaskAttemptContext is not serializable resulting in the confBytes construct + * @note CombineFileSplit is not serializable resulting in the splitBytes construct */ -class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, index: Integer) +class PortableDataStream(@transient isplit: CombineFileSplit, @transient context: TaskAttemptContext, index: Integer) extends Serializable { - // transient forces file to be reopened after being moved (serialization) + // transient forces file to be reopened after being serialization + // it is also used for non-serializable classes + @transient private var fileIn: FSDataInputStream = null.asInstanceOf[FSDataInputStream] @transient private var isOpen = false + + private val confBytes = { + val baos = new ByteArrayOutputStream() + context.getConfiguration.write(new DataOutputStream(baos)) + baos.toByteArray + } + + private val splitBytes = { + val baos = new ByteArrayOutputStream() + isplit.write(new DataOutputStream(baos)) + baos.toByteArray + } + + @transient + private lazy val split = { + val bais = new ByteArrayInputStream(splitBytes) + val nsplit = new CombineFileSplit() + nsplit.readFields(new DataInputStream(bais)) + nsplit + } + + @transient + private lazy val conf = { + val bais = new ByteArrayInputStream(confBytes) + val nconf = new Configuration() + nconf.readFields(new DataInputStream(bais)) + nconf + } /** * Calculate the path name independently of opening the file */ + @transient private lazy val path = { val pathp = split.getPath(index) pathp.toString @@ -80,7 +114,7 @@ class PortableDataStream(split: CombineFileSplit, context: TaskAttemptContext, i def open(): FSDataInputStream = { if (!isOpen) { val pathp = split.getPath(index) - val fs = pathp.getFileSystem(context.getConfiguration) + val fs = pathp.getFileSystem(conf) fileIn = fs.open(pathp) isOpen=true } @@ -207,4 +241,4 @@ abstract class BinaryRecordReader[T]( parseByteArray(innerBuffer) } def parseByteArray(inArray: Array[Byte]): T -} \ No newline at end of file +} diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b4dc8de323c1d..18566e633ce8e 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -881,7 +881,6 @@ public void binaryFilesCaching() throws Exception { readRDD.foreach(new VoidFunction>() { @Override public void call(Tuple2 stringPortableDataStreamTuple2) throws Exception { - stringPortableDataStreamTuple2._2().getPath(); stringPortableDataStreamTuple2._2().toArray(); // force the file to read } }); diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index d2913c7d2d479..c58072d4a1dc3 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -280,6 +280,43 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(indata.toArray === testOutput) } + test("portabledatastream flatmap tests") { + sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1,2,3,4,5,6) + val numOfCopies = 3 + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName) + val mappedRdd = inRdd.map{ + curData: (String, PortableDataStream) => + (curData._2.getPath(),curData._2) + } + val copyRdd = mappedRdd.flatMap{ + curData: (String, PortableDataStream) => + for(i <- 1 to numOfCopies) yield (i,curData._2) + } + + val copyArr: Array[(Int, PortableDataStream)] = copyRdd.collect() + + // Try reading the output back as an object file + assert(copyArr.length == numOfCopies) + copyArr.foreach{ + cEntry: (Int, PortableDataStream) => + assert(cEntry._2.toArray === testOutput) + } + + } + test("fixed record length binary file as byte array") { // a fixed length of 6 bytes From 4163e38bccca33608fc4a241760e86d4862793b5 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 1 Oct 2014 13:47:53 +0200 Subject: [PATCH 18/29] fixing line length and output from FSDataInputStream to DataInputStream to minimize sensitivity to Hadoop API changes --- .../org/apache/spark/input/RawFileInput.scala | 59 ++++++++----------- 1 file changed, 26 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index 1f3989995373c..9c97f41097e52 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -18,25 +18,24 @@ package org.apache.spark.input import scala.collection.JavaConversions._ -import com.google.common.io.{ByteStreams, Closeables} +import com.google.common.io.{ ByteStreams, Closeables } import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.hadoop.fs.{FSDataInputStream, Path} +import org.apache.hadoop.fs.{ FSDataInputStream, Path } import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutputStream, DataInputStream} - +import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, DataOutputStream, DataInputStream } /** * A general format for reading whole files in as streams, byte arrays, * or other functions to be added */ abstract class StreamFileInputFormat[T] - extends CombineFileInputFormat[String,T] { + extends CombineFileInputFormat[String, T] { override protected def isSplitable(context: JobContext, file: Path): Boolean = false /** * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. @@ -51,8 +50,7 @@ abstract class StreamFileInputFormat[T] super.setMaxSplitSize(maxSplitSize) } - def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): - RecordReader[String,T] + def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): RecordReader[String, T] } @@ -62,13 +60,14 @@ abstract class StreamFileInputFormat[T] * @note TaskAttemptContext is not serializable resulting in the confBytes construct * @note CombineFileSplit is not serializable resulting in the splitBytes construct */ -class PortableDataStream(@transient isplit: CombineFileSplit, @transient context: TaskAttemptContext, index: Integer) +class PortableDataStream(@transient isplit: CombineFileSplit, + @transient context: TaskAttemptContext, index: Integer) extends Serializable { // transient forces file to be reopened after being serialization // it is also used for non-serializable classes @transient - private var fileIn: FSDataInputStream = null.asInstanceOf[FSDataInputStream] + private var fileIn: DataInputStream = null.asInstanceOf[DataInputStream] @transient private var isOpen = false @@ -111,12 +110,12 @@ class PortableDataStream(@transient isplit: CombineFileSplit, @transient context /** * create a new DataInputStream from the split and context */ - def open(): FSDataInputStream = { + def open(): DataInputStream = { if (!isOpen) { val pathp = split.getPath(index) val fs = pathp.getFileSystem(conf) fileIn = fs.open(pathp) - isOpen=true + isOpen = true } fileIn } @@ -138,7 +137,7 @@ class PortableDataStream(@transient isplit: CombineFileSplit, @transient context if (isOpen) { try { fileIn.close() - isOpen=false + isOpen = false } catch { case ioe: java.io.IOException => // do nothing } @@ -152,20 +151,17 @@ class PortableDataStream(@transient isplit: CombineFileSplit, @transient context * to reading files out as streams */ abstract class StreamBasedRecordReader[T]( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) extends RecordReader[String, T] { - - // True means the current file has been processed, then skip it. private var processed = false private var key = "" private var value: T = null.asInstanceOf[T] - override def initialize(split: InputSplit, context: TaskAttemptContext) = {} override def close() = {} @@ -175,8 +171,6 @@ abstract class StreamBasedRecordReader[T]( override def getCurrentValue = value - - override def nextKeyValue = { if (!processed) { val fileIn = new PortableDataStream(split, context, index) @@ -202,9 +196,9 @@ abstract class StreamBasedRecordReader[T]( * Reads the record in directly as a stream for other objects to manipulate and handle */ private[spark] class StreamRecordReader( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) extends StreamBasedRecordReader[PortableDataStream](split, context, index) { def parseStream(inStream: PortableDataStream): PortableDataStream = inStream @@ -215,12 +209,11 @@ private[spark] class StreamRecordReader( * BinaryRecordReader (as Byte array) */ private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { - override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)= - { - new CombineFileRecordReader[String,PortableDataStream]( - split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader] - ) - } + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = + { + new CombineFileRecordReader[String, PortableDataStream]( + split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader]) + } } /** @@ -229,10 +222,10 @@ private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDat * the file as a byte array */ abstract class BinaryRecordReader[T]( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends StreamBasedRecordReader[T](split,context,index) { + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends StreamBasedRecordReader[T](split, context, index) { def parseStream(inpStream: PortableDataStream): T = { val inStream = inpStream.open() From 05887379eafdc359206753a68571aaf3fb2dd7a6 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 1 Oct 2014 14:50:06 +0200 Subject: [PATCH 19/29] filename check in "binary file input as byte array" test now ignores prefixes and suffixes which might get added by Hadoop --- core/src/test/scala/org/apache/spark/FileSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index c58072d4a1dc3..e1542eb04d9da 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -245,7 +245,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val (infile: String, indata: PortableDataStream) = inRdd.first // Try reading the output back as an object file - assert(infile === outFileName) + assert(outFileName.contains(infile)) // a prefix may get added assert(indata.toArray === testOutput) } From b348ce1eba4117e0d10479373da175688a612204 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 1 Oct 2014 15:50:20 +0200 Subject: [PATCH 20/29] fixed order in check (prefix only appears on jenkins not when I run unit tests locally) --- core/src/test/scala/org/apache/spark/FileSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index e1542eb04d9da..a2e95fdc84dc3 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -245,7 +245,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val (infile: String, indata: PortableDataStream) = inRdd.first // Try reading the output back as an object file - assert(outFileName.contains(infile)) // a prefix may get added + assert(infile.contains(outFileName)) // a prefix may get added assert(indata.toArray === testOutput) } From c27a8f144eeb4e1c03d614691efae0241cca6fab Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 1 Oct 2014 16:20:05 +0200 Subject: [PATCH 21/29] jenkins crashed before running anything last time, so making minor change --- core/src/test/scala/org/apache/spark/FileSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index a2e95fdc84dc3..e1c33d679fd24 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -244,7 +244,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val inRdd = sc.binaryFiles(outFileName) val (infile: String, indata: PortableDataStream) = inRdd.first - // Try reading the output back as an object file + // Make sure the name and array match assert(infile.contains(outFileName)) // a prefix may get added assert(indata.toArray === testOutput) } From 49174d90f5f623653421a2901b58a5b2116ef08c Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 2 Oct 2014 14:33:34 +0200 Subject: [PATCH 22/29] removed unneeded classes added DeveloperApi note to portabledatastreams since the implementation might change --- .../org/apache/spark/input/RawFileInput.scala | 24 +++---------------- 1 file changed, 3 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index 9c97f41097e52..e1caf9c44ec16 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext import org.apache.hadoop.fs.{ FSDataInputStream, Path } +import org.apache.spark.annotation.DeveloperApi import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader @@ -60,6 +61,7 @@ abstract class StreamFileInputFormat[T] * @note TaskAttemptContext is not serializable resulting in the confBytes construct * @note CombineFileSplit is not serializable resulting in the splitBytes construct */ +@DeveloperApi class PortableDataStream(@transient isplit: CombineFileSplit, @transient context: TaskAttemptContext, index: Integer) extends Serializable { @@ -205,8 +207,7 @@ private[spark] class StreamRecordReader( } /** - * A class for extracting the information from the file using the - * BinaryRecordReader (as Byte array) + * The format for the PortableDataStream files */ private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = @@ -216,22 +217,3 @@ private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDat } } -/** - * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single binary file - * out in a key-value pair, where the key is the file path and the value is the entire content of - * the file as a byte array - */ -abstract class BinaryRecordReader[T]( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends StreamBasedRecordReader[T](split, context, index) { - - def parseStream(inpStream: PortableDataStream): T = { - val inStream = inpStream.open() - val innerBuffer = ByteStreams.toByteArray(inStream) - Closeables.close(inStream, false) - parseByteArray(innerBuffer) - } - def parseByteArray(inArray: Array[Byte]): T -} From a32fef7b4905ef098be9e4f73e15ebdfea6a545b Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 2 Oct 2014 14:41:19 +0200 Subject: [PATCH 23/29] removed unneeded classes added DeveloperApi note to portabledatastreams since the implementation might change --- core/src/main/scala/org/apache/spark/input/RawFileInput.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala index e1caf9c44ec16..ed353445bf486 100644 --- a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala +++ b/core/src/main/scala/org/apache/spark/input/RawFileInput.scala @@ -35,7 +35,7 @@ import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, DataOutputStream, * A general format for reading whole files in as streams, byte arrays, * or other functions to be added */ -abstract class StreamFileInputFormat[T] +private[spark] abstract class StreamFileInputFormat[T] extends CombineFileInputFormat[String, T] { override protected def isSplitable(context: JobContext, file: Path): Boolean = false /** @@ -152,7 +152,7 @@ class PortableDataStream(@transient isplit: CombineFileSplit, * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] * to reading files out as streams */ -abstract class StreamBasedRecordReader[T]( +private[spark] abstract class StreamBasedRecordReader[T]( split: CombineFileSplit, context: TaskAttemptContext, index: Integer) From 92bda0daf2fffeea0f1de9199fc71fe978a165c7 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Mon, 20 Oct 2014 19:17:18 +0200 Subject: [PATCH 24/29] added new tests, renamed files, fixed several of the javaapi functions, formatted code more nicely --- .../spark/api/java/JavaSparkContext.scala | 5 +- .../input/FixedLengthBinaryInputFormat.scala | 10 ---- .../input/FixedLengthBinaryRecordReader.scala | 18 ------ ...leInput.scala => PortableDataStream.scala} | 0 .../org/apache/spark/rdd/BinaryFileRDD.scala | 5 +- .../java/org/apache/spark/JavaAPISuite.java | 4 +- .../scala/org/apache/spark/FileSuite.scala | 60 +++++++++++++++++++ 7 files changed, 66 insertions(+), 36 deletions(-) rename core/src/main/scala/org/apache/spark/input/{RawFileInput.scala => PortableDataStream.scala} (100%) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index ef107b27c87fb..7403a0176bb5f 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -256,9 +256,12 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * * @param minPartitions A suggestion value of the minimal splitting number for input data. */ - def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): + 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)) + /** * Read a directory of files as DataInputStream from HDFS, * a local file system (available on all nodes), or any Hadoop-supported file system URI diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 646fe23738a66..852be54d181d4 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -29,7 +29,6 @@ import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAt */ private[spark] object FixedLengthBinaryInputFormat { - /** * This function retrieves the recordLength by checking the configuration parameter * @@ -39,13 +38,10 @@ private[spark] object FixedLengthBinaryInputFormat { // 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 */ @@ -60,7 +56,6 @@ private[spark] class FixedLengthBinaryInputFormat } else { true } - } /** @@ -69,14 +64,11 @@ private[spark] class FixedLengthBinaryInputFormat * 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 { @@ -91,7 +83,5 @@ private[spark] class FixedLengthBinaryInputFormat RecordReader[LongWritable, BytesWritable] = { new FixedLengthBinaryRecordReader } - var recordLength = -1 - } diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index eb27a98fe09f2..98988910831be 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -76,66 +76,48 @@ private[spark] class FixedLengthBinaryRecordReader // 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 diff --git a/core/src/main/scala/org/apache/spark/input/RawFileInput.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/input/RawFileInput.scala rename to core/src/main/scala/org/apache/spark/input/PortableDataStream.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index c7dc50820d59b..3d1b5f1b543f5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -17,14 +17,11 @@ package org.apache.spark.rdd -/** Allows better control of the partitioning - * - */ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{InterruptibleIterator, TaskContext, Partition, SparkContext} import org.apache.spark.input.StreamFileInputFormat +import org.apache.spark.{Partition, SparkContext} private[spark] class BinaryFileRDD[T]( sc : SparkContext, diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 18566e633ce8e..a39ee903ba3c9 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -844,7 +844,6 @@ public void binaryFiles() throws Exception { // Reusing the wholeText files example byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); - String tempDirName = tempDir.getAbsolutePath(); File file1 = new File(tempDirName + "/part-00000"); @@ -866,7 +865,6 @@ public void binaryFilesCaching() throws Exception { // Reusing the wholeText files example byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); - String tempDirName = tempDir.getAbsolutePath(); File file1 = new File(tempDirName + "/part-00000"); @@ -877,7 +875,7 @@ public void binaryFilesCaching() throws Exception { channel1.write(bbuf); channel1.close(); - JavaPairRDD readRDD = sc.binaryFiles(tempDirName,3).cache(); + JavaPairRDD readRDD = sc.binaryFiles(tempDirName).cache(); readRDD.foreach(new VoidFunction>() { @Override public void call(Tuple2 stringPortableDataStreamTuple2) throws Exception { diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index e1c33d679fd24..e265b43fb79d1 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import java.io.{File, FileWriter} import org.apache.spark.input.PortableDataStream +import org.apache.spark.storage.StorageLevel import scala.io.Source @@ -280,6 +281,37 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(indata.toArray === testOutput) } + test("portabledatastream persist disk storage") { + sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1,2,3,4,5,6) + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName).persist(StorageLevel.DISK_ONLY) + inRdd.foreach{ + curData: (String, PortableDataStream) => + curData._2.toArray() // force the file to read + } + val mappedRdd = inRdd.map{ + curData: (String, PortableDataStream) => + (curData._2.getPath(),curData._2) + } + val (infile: String, indata: PortableDataStream) = mappedRdd.first + + // Try reading the output back as an object file + + assert(indata.toArray === testOutput) + } + test("portabledatastream flatmap tests") { sc = new SparkContext("local", "test") val outFile = new File(tempDir, "record-bytestream-00000.bin") @@ -348,6 +380,34 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(indata === testOutput) } + test ("negative binary record length should raise an exception") { + // a fixed length of 6 bytes + sc = new SparkContext("local", "test") + + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1,2,3,4,5,6) + val testOutputCopies = 10 + + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + for(i <- 1 to testOutputCopies) { + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + channel.write(bbuf) + } + channel.close() + file.close() + + val inRdd = sc.binaryRecords(outFileName, -1) + + intercept[SparkException] { + inRdd.count + } + } + test("file caching") { sc = new SparkContext("local", "test") val out = new FileWriter(tempDir + "/input") From 8ac288bc09e779f1b4c96dcb497ee4eca962439f Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Tue, 21 Oct 2014 07:54:17 +0200 Subject: [PATCH 25/29] fixed a single slightly over 100 character line --- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 7403a0176bb5f..3ff95d41de998 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -260,7 +260,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork JavaPairRDD[String,PortableDataStream] = new JavaPairRDD(sc.binaryFiles(path,minPartitions)) def binaryFiles(path: String): - JavaPairRDD[String,PortableDataStream] = new JavaPairRDD(sc.binaryFiles(path,defaultMinPartitions)) + JavaPairRDD[String,PortableDataStream] = + new JavaPairRDD(sc.binaryFiles(path,defaultMinPartitions)) /** * Read a directory of files as DataInputStream from HDFS, From 7b9d181f78c5fed674702573b1b0f69a183d88fa Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 22 Oct 2014 00:14:03 +0200 Subject: [PATCH 26/29] removing developer API, cleaning up imports --- .../spark/input/PortableDataStream.scala | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index ed353445bf486..dcb74f3556979 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -17,19 +17,15 @@ package org.apache.spark.input -import scala.collection.JavaConversions._ -import com.google.common.io.{ ByteStreams, Closeables } -import org.apache.hadoop.mapreduce.InputSplit +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import com.google.common.io.ByteStreams import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit -import org.apache.hadoop.mapreduce.RecordReader -import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.hadoop.fs.{ FSDataInputStream, Path } -import org.apache.spark.annotation.DeveloperApi -import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat -import org.apache.hadoop.mapreduce.JobContext -import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader -import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, DataOutputStream, DataInputStream } +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat, CombineFileRecordReader, CombineFileSplit} + +import scala.collection.JavaConversions._ /** * A general format for reading whole files in as streams, byte arrays, @@ -61,7 +57,6 @@ private[spark] abstract class StreamFileInputFormat[T] * @note TaskAttemptContext is not serializable resulting in the confBytes construct * @note CombineFileSplit is not serializable resulting in the splitBytes construct */ -@DeveloperApi class PortableDataStream(@transient isplit: CombineFileSplit, @transient context: TaskAttemptContext, index: Integer) extends Serializable { From 6379be487cfa91097f3591dfd05b8e87e09c2399 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Wed, 22 Oct 2014 00:15:16 +0200 Subject: [PATCH 27/29] reorganizing code --- .../spark/input/PortableDataStream.scala | 138 +++++++++--------- 1 file changed, 69 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index dcb74f3556979..9c8121e2a6d14 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -51,6 +51,75 @@ private[spark] abstract class StreamFileInputFormat[T] } +/** + * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] + * to reading files out as streams + */ +private[spark] abstract class StreamBasedRecordReader[T]( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends RecordReader[String, T] { + + // True means the current file has been processed, then skip it. + private var processed = false + + private var key = "" + private var value: T = null.asInstanceOf[T] + + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + override def close() = {} + + override def getProgress = if (processed) 1.0f else 0.0f + + override def getCurrentKey = key + + override def getCurrentValue = value + + override def nextKeyValue = { + if (!processed) { + val fileIn = new PortableDataStream(split, context, index) + value = parseStream(fileIn) + fileIn.close() // if it has not been open yet, close does nothing + key = fileIn.getPath + processed = true + true + } else { + false + } + } + + /** + * Parse the stream (and close it afterwards) and return the value as in type T + * @param inStream the stream to be read in + * @return the data formatted as + */ + def parseStream(inStream: PortableDataStream): T +} + +/** + * Reads the record in directly as a stream for other objects to manipulate and handle + */ +private[spark] class StreamRecordReader( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends StreamBasedRecordReader[PortableDataStream](split, context, index) { + + def parseStream(inStream: PortableDataStream): PortableDataStream = inStream +} + +/** + * The format for the PortableDataStream files + */ +private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = + { + new CombineFileRecordReader[String, PortableDataStream]( + split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader]) + } +} + /** * A class that allows DataStreams to be serialized and moved around by not creating them * until they need to be read @@ -143,72 +212,3 @@ class PortableDataStream(@transient isplit: CombineFileSplit, def getPath(): String = path } -/** - * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] - * to reading files out as streams - */ -private[spark] abstract class StreamBasedRecordReader[T]( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends RecordReader[String, T] { - - // True means the current file has been processed, then skip it. - private var processed = false - - private var key = "" - private var value: T = null.asInstanceOf[T] - - override def initialize(split: InputSplit, context: TaskAttemptContext) = {} - override def close() = {} - - override def getProgress = if (processed) 1.0f else 0.0f - - override def getCurrentKey = key - - override def getCurrentValue = value - - override def nextKeyValue = { - if (!processed) { - val fileIn = new PortableDataStream(split, context, index) - value = parseStream(fileIn) - fileIn.close() // if it has not been open yet, close does nothing - key = fileIn.getPath - processed = true - true - } else { - false - } - } - - /** - * Parse the stream (and close it afterwards) and return the value as in type T - * @param inStream the stream to be read in - * @return the data formatted as - */ - def parseStream(inStream: PortableDataStream): T -} - -/** - * Reads the record in directly as a stream for other objects to manipulate and handle - */ -private[spark] class StreamRecordReader( - split: CombineFileSplit, - context: TaskAttemptContext, - index: Integer) - extends StreamBasedRecordReader[PortableDataStream](split, context, index) { - - def parseStream(inStream: PortableDataStream): PortableDataStream = inStream -} - -/** - * The format for the PortableDataStream files - */ -private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { - override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = - { - new CombineFileRecordReader[String, PortableDataStream]( - split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader]) - } -} - From 359a0960d7122d374a99c86e201cf21d993f3886 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 30 Oct 2014 11:43:27 +1100 Subject: [PATCH 28/29] making the final corrections suggested by @mateiz and renaming a few functions to make their usage clearer --- .../scala/org/apache/spark/SparkContext.scala | 1 - .../spark/api/java/JavaSparkContext.scala | 33 ------------------- .../spark/input/PortableDataStream.scala | 5 +-- .../org/apache/spark/rdd/BinaryFileRDD.scala | 22 ++++++------- .../scala/org/apache/spark/FileSuite.scala | 8 ++--- 5 files changed, 17 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 80490fecd978b..d6a4cd42f7604 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -518,7 +518,6 @@ class SparkContext(config: SparkConf) extends Logging { * * @param minPartitions A suggestion value of the minimal splitting number for input data. * - * @note Care must be taken to close the files afterwards * @note Small files are preferred, large file is also allowable, but may cause bad performance. */ @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 3ff95d41de998..373e6f1d12864 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -263,39 +263,6 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork JavaPairRDD[String,PortableDataStream] = new JavaPairRDD(sc.binaryFiles(path,defaultMinPartitions)) - /** - * 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. - * - *

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 rdd = sparkContext.binaryFiles("hdfs://a-hdfs-path")`, - * - *

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 binaryArrays(path: String, minPartitions: Int = defaultMinPartitions): - JavaPairRDD[String, Array[Byte]] = - new JavaPairRDD(sc.binaryFiles(path,minPartitions).mapValues(_.toArray())) - /** * 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 diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 9c8121e2a6d14..9cfe3e5c1742a 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -35,9 +35,10 @@ private[spark] abstract class StreamFileInputFormat[T] extends CombineFileInputFormat[String, T] { override protected def isSplitable(context: JobContext, file: Path): Boolean = false /** - * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. + * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API + * which is set through setMaxSplitSize */ - def setMaxSplitSize(context: JobContext, minPartitions: Int) { + def setMinPartitions(context: JobContext, minPartitions: Int) { val files = listStatus(context) val totalLen = files.map { file => if (file.isDir) 0L else file.getLen diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index 3d1b5f1b543f5..c01196de6bad7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -17,22 +17,21 @@ package org.apache.spark.rdd -import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.conf.{ Configurable, Configuration } import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.input.StreamFileInputFormat -import org.apache.spark.{Partition, SparkContext} +import org.apache.spark.{ Partition, SparkContext } private[spark] class BinaryFileRDD[T]( - sc : SparkContext, - inputFormatClass: Class[_ <: StreamFileInputFormat[T]], - keyClass: Class[String], - valueClass: Class[T], - @transient conf: Configuration, - minPartitions: Int) + sc: SparkContext, + inputFormatClass: Class[_ <: StreamFileInputFormat[T]], + keyClass: Class[String], + valueClass: Class[T], + @transient conf: Configuration, + minPartitions: Int) extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { - override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance inputFormat match { @@ -41,13 +40,12 @@ private[spark] class BinaryFileRDD[T]( case _ => } val jobContext = newJobContext(conf, jobId) - inputFormat.setMaxSplitSize(jobContext, minPartitions) + inputFormat.setMinPartitions(jobContext, minPartitions) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) { result(i) = new NewHadoopPartition( - id, i, rawSplits(i).asInstanceOf[InputSplit with Writable] - ) + id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) } result } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index e265b43fb79d1..3f4c7b0180486 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -243,7 +243,7 @@ class FileSuite extends FunSuite with LocalSparkContext { file.close() val inRdd = sc.binaryFiles(outFileName) - val (infile: String, indata: PortableDataStream) = inRdd.first + val (infile: String, indata: PortableDataStream) = inRdd.collect.head // Make sure the name and array match assert(infile.contains(outFileName)) // a prefix may get added @@ -274,7 +274,7 @@ class FileSuite extends FunSuite with LocalSparkContext { curData: (String, PortableDataStream) => (curData._2.getPath(),curData._2) } - val (infile: String, indata: PortableDataStream) = mappedRdd.first + val (infile: String, indata: PortableDataStream) = mappedRdd.collect.head // Try reading the output back as an object file @@ -305,7 +305,7 @@ class FileSuite extends FunSuite with LocalSparkContext { curData: (String, PortableDataStream) => (curData._2.getPath(),curData._2) } - val (infile: String, indata: PortableDataStream) = mappedRdd.first + val (infile: String, indata: PortableDataStream) = mappedRdd.collect.head // Try reading the output back as an object file @@ -376,7 +376,7 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(inRdd.count == testOutputCopies) // now just compare the first one - val indata: Array[Byte] = inRdd.first + val indata: Array[Byte] = inRdd.collect.head assert(indata === testOutput) } From 3c49a305033d76e07bca60a72600c0db544407dd Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Thu, 30 Oct 2014 11:49:15 +1100 Subject: [PATCH 29/29] fixing wholetextfileinput to it has the same setMinPartitions function as in BinaryData files --- .../org/apache/spark/input/WholeTextFileInputFormat.scala | 5 +++-- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index 4cb450577796a..183bce3d8d8d3 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -48,9 +48,10 @@ private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[Str } /** - * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. + * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API, + * which is set through setMaxSplitSize */ - def setMaxSplitSize(context: JobContext, minPartitions: Int) { + def setMinPartitions(context: JobContext, minPartitions: Int) { val files = listStatus(context) val totalLen = files.map { file => if (file.isDir) 0L else file.getLen diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index f2b3a64bf1345..26bfb2d5fa8a3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -182,7 +182,7 @@ private[spark] class WholeTextFileRDD( case _ => } val jobContext = newJobContext(conf, jobId) - inputFormat.setMaxSplitSize(jobContext, minPartitions) + inputFormat.setMinPartitions(jobContext, minPartitions) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) {