forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
adding binary and byte file support spark
- Loading branch information
Showing
3 changed files
with
171 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
102 changes: 102 additions & 0 deletions
102
core/src/main/scala/org/apache/spark/input/BinaryFileInput.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
} |
48 changes: 48 additions & 0 deletions
48
core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
} | ||
} |