Skip to content
Snippets Groups Projects
Commit a65766bf authored by jerryshao's avatar jerryshao Committed by Sean Owen
Browse files

[SPARK-5826][Streaming] Fix Configuration not serializable problem

Author: jerryshao <saisai.shao@intel.com>

Closes #4612 from jerryshao/SPARK-5826 and squashes the following commits:

7ec71db [jerryshao] Remove transient for conf statement
88d84e6 [jerryshao] Fix Configuration not serializable problem
parent c06e42f2
No related branches found
No related tags found
No related merge requests found
......@@ -18,7 +18,6 @@
package org.apache.spark.streaming.dstream
import java.io.{IOException, ObjectInputStream}
import java.util.concurrent.ConcurrentHashMap
import scala.collection.mutable
import scala.reflect.ClassTag
......@@ -27,6 +26,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.spark.SerializableWritable
import org.apache.spark.rdd.{RDD, UnionRDD}
import org.apache.spark.streaming._
import org.apache.spark.util.{TimeStampedHashMap, Utils}
......@@ -78,6 +78,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]](
(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F])
extends InputDStream[(K, V)](ssc_) {
private val serializableConfOpt = conf.map(new SerializableWritable(_))
// This is a def so that it works during checkpoint recovery:
private def clock = ssc.scheduler.clock
......@@ -240,7 +242,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]](
/** Generate one RDD from an array of files */
private def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
val fileRDDs = files.map(file =>{
val rdd = conf match {
val rdd = serializableConfOpt.map(_.value) match {
case Some(config) => context.sparkContext.newAPIHadoopFile(
file,
fm.runtimeClass.asInstanceOf[Class[F]],
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment