aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorpierre-borckmans <pierre.borckmans@realimpactanalytics.com>2015-12-24 13:48:21 +0000
committerSean Owen <sowen@cloudera.com>2015-12-24 13:48:21 +0000
commitea4aab7e87fbcf9ac90f93af79cc892b56508aa0 (patch)
tree3f53922ecfdb38969a6385f0660fa9ff26991e9a /core
parent502476e45c314a1229b3bce1c61f5cb94a9fc04b (diff)
downloadspark-ea4aab7e87fbcf9ac90f93af79cc892b56508aa0.tar.gz
spark-ea4aab7e87fbcf9ac90f93af79cc892b56508aa0.tar.bz2
spark-ea4aab7e87fbcf9ac90f93af79cc892b56508aa0.zip
[SPARK-12440][CORE] Avoid setCheckpoint warning when directory is not local
In SparkContext method `setCheckpointDir`, a warning is issued when spark master is not local and the passed directory for the checkpoint dir appears to be local. In practice, when relying on HDFS configuration file and using a relative path for the checkpoint directory (using an incomplete URI without HDFS scheme, ...), this warning should not be issued and might be confusing. In fact, in this case, the checkpoint directory is successfully created, and the checkpointing mechanism works as expected. This PR uses the `FileSystem` instance created with the given directory, and checks whether it is local or not. (The rationale is that since this same `FileSystem` instance is used to create the checkpoint dir anyway and can therefore be reliably used to determine if it is local or not). The warning is only issued if the directory is not local, on top of the existing conditions. Author: pierre-borckmans <pierre.borckmans@realimpactanalytics.com> Closes #10392 from pierre-borckmans/SPARK-12440_CheckpointDir_Warning_NonLocal.
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala5
1 files changed, 3 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 67230f4207..d506782b73 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -2073,8 +2073,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
// its own local file system, which is incorrect because the checkpoint files
// are actually on the executor machines.
if (!isLocal && Utils.nonLocalPaths(directory).isEmpty) {
- logWarning("Checkpoint directory must be non-local " +
- "if Spark is running on a cluster: " + directory)
+ logWarning("Spark is not running in local mode, therefore the checkpoint directory " +
+ s"must not be on the local filesystem. Directory '$directory' " +
+ "appears to be on the local filesystem.")
}
checkpointDir = Option(directory).map { dir =>