aboutsummaryrefslogtreecommitdiff
path: root/mllib/src/test
diff options
context:
space:
mode:
authorLianhui Wang <lianhuiwang09@gmail.com>2016-06-01 08:30:38 -0500
committerSean Owen <sowen@cloudera.com>2016-06-01 08:30:38 -0500
commit6563d72b168c39115376e73788b48a2d60803d4e (patch)
treeaf6ffec045d492385f83036598f73f748bce7728 /mllib/src/test
parente4ce1bc4f3ca088365ff199e563f23a552dc88ef (diff)
downloadspark-6563d72b168c39115376e73788b48a2d60803d4e.tar.gz
spark-6563d72b168c39115376e73788b48a2d60803d4e.tar.bz2
spark-6563d72b168c39115376e73788b48a2d60803d4e.zip
[SPARK-15664][MLLIB] Replace FileSystem.get(conf) with path.getFileSystem(conf) when removing CheckpointFile in MLlib
## What changes were proposed in this pull request? if sparkContext.set CheckpointDir to another Dir that is not default FileSystem, it will throw exception when removing CheckpointFile in MLlib. So we should always get the FileSystem from Path to avoid wrong FS problem. ## How was this patch tested? N/A Author: Lianhui Wang <lianhuiwang09@gmail.com> Closes #13408 from lianhuiwang/SPARK-15664.
Diffstat (limited to 'mllib/src/test')
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala8
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala8
2 files changed, 10 insertions, 6 deletions
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala
index e331c75989..a13e7f63a9 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark.mllib.impl
-import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.Path
import org.apache.spark.{SparkContext, SparkFunSuite}
import org.apache.spark.graphx.{Edge, Graph}
@@ -140,9 +140,11 @@ private object PeriodicGraphCheckpointerSuite {
// Instead, we check for the presence of the checkpoint files.
// This test should continue to work even after this graph.isCheckpointed issue
// is fixed (though it can then be simplified and not look for the files).
- val fs = FileSystem.get(graph.vertices.sparkContext.hadoopConfiguration)
+ val hadoopConf = graph.vertices.sparkContext.hadoopConfiguration
graph.getCheckpointFiles.foreach { checkpointFile =>
- assert(!fs.exists(new Path(checkpointFile)),
+ val path = new Path(checkpointFile)
+ val fs = path.getFileSystem(hadoopConf)
+ assert(!fs.exists(path),
"Graph checkpoint file should have been removed")
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala
index b2a459a68b..14adf8c29f 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark.mllib.impl
-import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.Path
import org.apache.spark.{SparkContext, SparkFunSuite}
import org.apache.spark.mllib.util.MLlibTestSparkContext
@@ -127,9 +127,11 @@ private object PeriodicRDDCheckpointerSuite {
// Instead, we check for the presence of the checkpoint files.
// This test should continue to work even after this rdd.isCheckpointed issue
// is fixed (though it can then be simplified and not look for the files).
- val fs = FileSystem.get(rdd.sparkContext.hadoopConfiguration)
+ val hadoopConf = rdd.sparkContext.hadoopConfiguration
rdd.getCheckpointFile.foreach { checkpointFile =>
- assert(!fs.exists(new Path(checkpointFile)), "RDD checkpoint file should have been removed")
+ val path = new Path(checkpointFile)
+ val fs = path.getFileSystem(hadoopConf)
+ assert(!fs.exists(path), "RDD checkpoint file should have been removed")
}
}