diff options
-rw-r--r-- | core/src/main/scala/spark/rdd/HadoopRDD.scala | 4 | ||||
-rw-r--r-- | core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 3 |
2 files changed, 6 insertions, 1 deletions
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 6c41b97780..e512423fd6 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.util.ReflectionUtils import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext} import spark.util.NextIterator -import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.conf.{Configuration, Configurable} /** @@ -132,4 +132,6 @@ class HadoopRDD[K, V]( override def checkpoint() { // Do nothing. Hadoop RDD should not be checkpointed. } + + def getConf: Configuration = confBroadcast.value.value } diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 184685528e..b1877dc06e 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -120,4 +120,7 @@ class NewHadoopRDD[K, V]( val theSplit = split.asInstanceOf[NewHadoopPartition] theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") } + + def getConf: Configuration = confBroadcast.value.value } + |