Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
43 changes: 43 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,9 @@ import scala.collection.mutable.HashMap
import scala.reflect.{ClassTag, classTag}
import scala.util.control.NonFatal

import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.avro.mapred.{AvroJob, AvroWrapper, AvroInputFormat}
import org.apache.commons.lang.SerializationUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
Expand Down Expand Up @@ -812,6 +815,46 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs)
}

/**
* Reads in a directory of Avro files from HDFS, a local file system (available on all nodes), or
* any Hadoop-supported file system URI. The records are read in as Generic Avro records. This
* also allows a user to register a schema with Kryo, if they so choose to. Users can also specify
* multiple schemas if they need to.
*
* You can do the following if you know the schema ahead of time:
* {{{
* val schema = new Schema.Parser().parse(schemaString)
* sc.avroFile("/input-path", schema)
* }}}
*
* or just:
* {{{
* sc.avroFile("/input-path")
* }}}
*/
def avroFile(path: String, schema: Schema): RDD[GenericRecord] = {
conf.registerAvroSchemas(schema)
hadoopFile[AvroWrapper[GenericRecord], NullWritable, AvroInputFormat[GenericRecord]](path)
.map(_._1.datum).setName(path)
}

/**
* Reads in a directory of Avro files from HDFS, a local file system (availavble on all nodes), or
* any Hadoop-supported file system URI. The second parameter determines what type of RDD is
* created. This is used for Specific or Reflect Avro records.
*
* {{{
* sc.avroFile("/input-path", classOf[CustomSpecificRecord])
* }}}
*/
def avroFile[T: ClassTag](path: String, schema: Class[T]): RDD[T] = {
val jobConf = new JobConf()
FileInputFormat.setInputPaths(jobConf, path)
AvroJob.setInputReflect(jobConf)
hadoopRDD(jobConf, classOf[AvroInputFormat[T]], classOf[AvroWrapper[T]], classOf[NullWritable])
.map(_._1.datum)
}

/**
* Read a text file from HDFS, a local file system (available on all nodes), or any
* Hadoop-supported file system URI, and return it as an RDD of Strings.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer}
import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator}
import org.apache.avro.generic.{GenericData, GenericRecord}
import org.apache.avro.util.Utf8
import org.roaringbitmap.{ArrayContainer, BitmapContainer, RoaringArray, RoaringBitmap}

import org.apache.spark._
Expand Down Expand Up @@ -375,7 +376,8 @@ private[serializer] object KryoSerializer {
classOf[Array[Short]],
classOf[Array[Long]],
classOf[BoundedPriorityQueue[_]],
classOf[SparkConf]
classOf[SparkConf],
classOf[Utf8]
)
}

Expand Down
Loading