diff options
author | Josh Rosen <joshrosen@databricks.com> | 2015-02-13 17:45:31 -0800 |
---|---|---|
committer | Patrick Wendell <patrick@databricks.com> | 2015-02-13 17:45:31 -0800 |
commit | d06d5ee9b33505774ef1e5becc01b47492f1a2dc (patch) | |
tree | a304948c135092ef296ad89226121adbc04452cb /core/src/test | |
parent | 4f4c6d5a5db04a56906bacdc85d7e5589b6edada (diff) | |
download | spark-d06d5ee9b33505774ef1e5becc01b47492f1a2dc.tar.gz spark-d06d5ee9b33505774ef1e5becc01b47492f1a2dc.tar.bz2 spark-d06d5ee9b33505774ef1e5becc01b47492f1a2dc.zip |
[SPARK-5227] [SPARK-5679] Disable FileSystem cache in WholeTextFileRecordReaderSuite
This patch fixes two difficult-to-reproduce Jenkins test failures in InputOutputMetricsSuite (SPARK-5227 and SPARK-5679). The problem was that WholeTextFileRecordReaderSuite modifies the `fs.local.block.size` Hadoop configuration and this change was affecting subsequent test suites due to Hadoop's caching of FileSystem instances (see HADOOP-8490 for more details).
The fix implemented here is to disable FileSystem caching in WholeTextFileRecordReaderSuite.
Author: Josh Rosen <joshrosen@databricks.com>
Closes #4599 from JoshRosen/inputoutputsuite-fix and squashes the following commits:
47dc447 [Josh Rosen] [SPARK-5227] [SPARK-5679] Disable FileSystem cache in WholeTextFileRecordReaderSuite
Diffstat (limited to 'core/src/test')
-rw-r--r-- | core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala | 12 |
1 files changed, 10 insertions, 2 deletions
diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 98b0a16ce8..2e58c159a2 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.Text -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.util.Utils import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, GzipCodec} @@ -42,7 +42,15 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { private var factory: CompressionCodecFactory = _ override def beforeAll() { - sc = new SparkContext("local", "test") + // Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which + // can cause Filesystem.get(Configuration) to return a cached instance created with a different + // configuration than the one passed to get() (see HADOOP-8490 for more details). This caused + // hard-to-reproduce test failures, since any suites that were run after this one would inherit + // the new value of "fs.local.block.size" (see SPARK-5227 and SPARK-5679). To work around this, + // we disable FileSystem caching in this suite. + val conf = new SparkConf().set("spark.hadoop.fs.file.impl.disable.cache", "true") + + sc = new SparkContext("local", "test", conf) // Set the block size of local file system to test whether files are split right or not. sc.hadoopConfiguration.setLong("fs.local.block.size", 32) |