aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorzsxwing <zsxwing@gmail.com>2015-02-06 11:50:20 -0800
committerAndrew Or <andrew@databricks.com>2015-02-06 11:51:09 -0800
commitaf2a2a263ac5d890e84d012b75fcb50e02c9ede8 (patch)
treece4ca74411f766879335a4914460003cc41f31aa /core
parentfb6c0cbac414f3c43dec2ab886ca8a1097b781f7 (diff)
downloadspark-af2a2a263ac5d890e84d012b75fcb50e02c9ede8.tar.gz
spark-af2a2a263ac5d890e84d012b75fcb50e02c9ede8.tar.bz2
spark-af2a2a263ac5d890e84d012b75fcb50e02c9ede8.zip
[SPARK-4361][Doc] Add more docs for Hadoop Configuration
I'm trying to point out reusing a Configuration in these APIs is dangerous. Any better idea? Author: zsxwing <zsxwing@gmail.com> Closes #3225 from zsxwing/SPARK-4361 and squashes the following commits: fe4e3d5 [zsxwing] Add more docs for Hadoop Configuration
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala20
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala28
2 files changed, 46 insertions, 2 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 24490fddc5..5623587c36 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -288,7 +288,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
// the bound port to the cluster manager properly
ui.foreach(_.bind())
- /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
+ /**
+ * A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse.
+ *
+ * '''Note:''' As it will be reused in all Hadoop RDDs, it's better not to modify it unless you
+ * plan to set some global configurations for all Hadoop RDDs.
+ */
val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf)
// Add each JAR given through the constructor
@@ -694,7 +699,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable),
* using the older MapReduce API (`org.apache.hadoop.mapred`).
*
- * @param conf JobConf for setting up the dataset
+ * @param conf JobConf for setting up the dataset. Note: This will be put into a Broadcast.
+ * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make
+ * sure you won't modify the conf. A safe approach is always creating a new conf for
+ * a new RDD.
* @param inputFormatClass Class of the InputFormat
* @param keyClass Class of the keys
* @param valueClass Class of the values
@@ -830,6 +838,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
*
+ * @param conf Configuration for setting up the dataset. Note: This will be put into a Broadcast.
+ * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make
+ * sure you won't modify the conf. A safe approach is always creating a new conf for
+ * a new RDD.
+ * @param fClass Class of the InputFormat
+ * @param kClass Class of the keys
+ * @param vClass Class of the values
+ *
* '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
* record, directly caching the returned RDD or directly passing it to an aggregation or shuffle
* operation will create many references to the same object.
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 97f5c9f257..6d6ed693be 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -373,6 +373,15 @@ class JavaSparkContext(val sc: SparkContext)
* other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
* etc).
*
+ * @param conf JobConf for setting up the dataset. Note: This will be put into a Broadcast.
+ * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make
+ * sure you won't modify the conf. A safe approach is always creating a new conf for
+ * a new RDD.
+ * @param inputFormatClass Class of the InputFormat
+ * @param keyClass Class of the keys
+ * @param valueClass Class of the values
+ * @param minPartitions Minimum number of Hadoop Splits to generate.
+ *
* '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
* record, directly caching the returned RDD will create many references to the same object.
* If you plan to directly cache Hadoop writable objects, you should first copy them using
@@ -395,6 +404,14 @@ class JavaSparkContext(val sc: SparkContext)
* Get an RDD for a Hadoop-readable dataset from a Hadooop JobConf giving its InputFormat and any
* other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
*
+ * @param conf JobConf for setting up the dataset. Note: This will be put into a Broadcast.
+ * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make
+ * sure you won't modify the conf. A safe approach is always creating a new conf for
+ * a new RDD.
+ * @param inputFormatClass Class of the InputFormat
+ * @param keyClass Class of the keys
+ * @param valueClass Class of the values
+ *
* '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
* record, directly caching the returned RDD will create many references to the same object.
* If you plan to directly cache Hadoop writable objects, you should first copy them using
@@ -476,6 +493,14 @@ class JavaSparkContext(val sc: SparkContext)
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
*
+ * @param conf Configuration for setting up the dataset. Note: This will be put into a Broadcast.
+ * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make
+ * sure you won't modify the conf. A safe approach is always creating a new conf for
+ * a new RDD.
+ * @param fClass Class of the InputFormat
+ * @param kClass Class of the keys
+ * @param vClass Class of the values
+ *
* '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
* record, directly caching the returned RDD will create many references to the same object.
* If you plan to directly cache Hadoop writable objects, you should first copy them using
@@ -675,6 +700,9 @@ class JavaSparkContext(val sc: SparkContext)
/**
* Returns the Hadoop configuration used for the Hadoop code (e.g. file systems) we reuse.
+ *
+ * '''Note:''' As it will be reused in all Hadoop RDDs, it's better not to modify it unless you
+ * plan to set some global configurations for all Hadoop RDDs.
*/
def hadoopConfiguration(): Configuration = {
sc.hadoopConfiguration