@@ -22,6 +22,7 @@ import java.io.{FileWriter, PrintWriter, File}
22
22
import org .apache .spark .SharedSparkContext
23
23
import org .apache .spark .deploy .SparkHadoopUtil
24
24
import org .apache .spark .scheduler .{SparkListenerTaskEnd , SparkListener }
25
+ import org .apache .spark .util .Utils
25
26
26
27
import org .scalatest .FunSuite
27
28
import org .scalatest .matchers .ShouldMatchers
@@ -108,26 +109,21 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with Shou
108
109
}
109
110
110
111
test(" exceptions while getting IO thread statistics should not fail tasks / jobs (SPARK-8062)" ) {
111
- // For some reason, the following code needs to be called in order for this regression test to
112
- // fail and reproduce the bug. The fact that this is necessary suggests that there may be other
113
- // bugs in our InputOutputMetrics code; SPARK-8086 tracks progress towards investigating this
114
- // issue, since fixing it is out of scope for SPARK-8062.
115
- val fs = FileSystem .getLocal(new Configuration ())
116
- val outPath = new Path (fs.getWorkingDirectory, " outdir" )
117
- SparkHadoopUtil .get.getFSBytesWrittenOnThreadCallback(outPath, fs.getConf)
112
+ val tempDir = Utils .createTempDir()
113
+ val outPath = new File (tempDir, " outfile" )
118
114
119
115
// Intentionally call this method with a null scheme, which will store an entry for a FileSystem
120
116
// with a null scheme into Hadoop's global `FileSystem.statisticsTable`.
121
117
FileSystem .getStatistics(null , classOf [FileSystem ])
122
118
123
119
// Prior to fixing SPARK-8062, this would fail with a NullPointerException in
124
120
// SparkHadoopUtil.getFileSystemThreadStatistics
125
- val rdd = sc.parallelize(Array (" a" , " b" , " c" , " d" ), 2 )
126
121
try {
122
+ val rdd = sc.parallelize(Array (" a" , " b" , " c" , " d" ), 2 )
127
123
rdd.saveAsTextFile(outPath.toString)
128
124
sc.textFile(outPath.toString).count()
129
125
} finally {
130
- fs.delete(outPath, true )
126
+ Utils .deleteRecursively(tempDir )
131
127
}
132
128
}
133
129
}
0 commit comments