spark-instrumented-optimizer/src/scala/spark/HadoopFile.scala

119 lines
3.6 KiB
Scala
Raw Normal View History

2010-03-29 19:17:55 -04:00
package spark
import mesos.SlaveOffer
2010-03-29 19:17:55 -04:00
import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.io.Text
import org.apache.hadoop.mapred.FileInputFormat
import org.apache.hadoop.mapred.InputFormat
2010-03-29 19:17:55 -04:00
import org.apache.hadoop.mapred.InputSplit
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapred.TextInputFormat
import org.apache.hadoop.mapred.RecordReader
import org.apache.hadoop.mapred.Reporter
import org.apache.hadoop.util.ReflectionUtils
2010-03-29 19:17:55 -04:00
/** A Spark split class that wraps around a Hadoop InputSplit */
2010-10-16 20:25:09 -04:00
@serializable class HadoopSplit(@transient s: InputSplit)
extends Split {
val inputSplit = new SerializableWritable[InputSplit](s)
2010-10-16 20:25:09 -04:00
// Hadoop gives each split a unique toString value, so use this as our ID
override def getId() = "HadoopSplit(" + inputSplit.toString + ")"
}
2010-03-29 19:17:55 -04:00
/**
* An RDD that reads a Hadoop file (from HDFS, S3, the local filesystem, etc)
* and represents it as a set of key-value pairs using a given InputFormat.
*/
class HadoopFile[K, V](
sc: SparkContext,
path: String,
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V])
extends RDD[(K, V)](sc) {
@transient val splits_ : Array[Split] = ConfigureLock.synchronized {
val conf = new JobConf()
FileInputFormat.setInputPaths(conf, path)
val inputFormat = createInputFormat(conf)
val inputSplits = inputFormat.getSplits(conf, sc.scheduler.numCores)
inputSplits.map(x => new HadoopSplit(x): Split).toArray
}
2010-03-29 19:17:55 -04:00
def createInputFormat(conf: JobConf): InputFormat[K, V] = {
ReflectionUtils.newInstance(inputFormatClass.asInstanceOf[Class[_]], conf)
.asInstanceOf[InputFormat[K, V]]
}
2010-03-29 19:17:55 -04:00
override def splits = splits_
2010-03-29 19:17:55 -04:00
override def iterator(theSplit: Split) = new Iterator[(K, V)] {
val split = theSplit.asInstanceOf[HadoopSplit]
var reader: RecordReader[K, V] = null
2010-03-29 19:17:55 -04:00
ConfigureLock.synchronized {
val conf = new JobConf()
val bufferSize = System.getProperty("spark.buffer.size", "65536")
conf.set("io.file.buffer.size", bufferSize)
val fmt = createInputFormat(conf)
reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL)
2010-03-29 19:17:55 -04:00
}
val key: K = keyClass.newInstance()
val value: V = valueClass.newInstance()
2010-03-29 19:17:55 -04:00
var gotNext = false
var finished = false
override def hasNext: Boolean = {
if (!gotNext) {
try {
finished = !reader.next(key, value)
} catch {
case eofe: java.io.EOFException =>
finished = true
}
2010-03-29 19:17:55 -04:00
gotNext = true
}
!finished
}
override def next: (K, V) = {
if (!gotNext) {
finished = !reader.next(key, value)
}
if (finished) {
throw new java.util.NoSuchElementException("End of stream")
}
2010-03-29 19:17:55 -04:00
gotNext = false
(key, value)
2010-03-29 19:17:55 -04:00
}
}
override def preferredLocations(split: Split) = {
// TODO: Filtering out "localhost" in case of file:// URLs
2010-10-16 20:25:09 -04:00
val hadoopSplit = split.asInstanceOf[HadoopSplit]
hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost")
}
2010-03-29 19:17:55 -04:00
}
/**
* Convenience class for Hadoop files read using TextInputFormat that
* represents the file as an RDD of Strings.
*/
class HadoopTextFile(sc: SparkContext, path: String)
extends MappedRDD[String, (LongWritable, Text)](
new HadoopFile(sc, path, classOf[TextInputFormat],
classOf[LongWritable], classOf[Text]),
{ pair: (LongWritable, Text) => pair._2.toString }
)
/**
* Object used to ensure that only one thread at a time is configuring Hadoop
* InputFormat classes. Apparently configuring them is not thread safe!
*/
2010-03-29 19:17:55 -04:00
object ConfigureLock {}