Skip to content

Commit 4a212fa

Browse files
committed
Standardize a bit more temp dir management
1 parent 9004081 commit 4a212fa

File tree

12 files changed

+43
-54
lines changed

12 files changed

+43
-54
lines changed

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -288,7 +288,7 @@ private[spark] object Utils extends Logging {
288288
} catch { case e: SecurityException => dir = null; }
289289
}
290290

291-
dir
291+
dir.getCanonicalFile
292292
}
293293

294294
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
1919

2020
import org.apache.spark.annotation.DeveloperApi
2121
import org.apache.spark.sql.catalyst.rules
22-
import org.apache.spark.sql.catalyst.util
22+
import org.apache.spark.util.Utils
2323

2424
/**
2525
* A collection of generators that build custom bytecode at runtime for performing the evaluation
@@ -52,7 +52,7 @@ package object codegen {
5252
@DeveloperApi
5353
object DumpByteCode {
5454
import scala.sys.process._
55-
val dumpDirectory = util.getTempFilePath("sparkSqlByteCode")
55+
val dumpDirectory = Utils.createTempDir()
5656
dumpDirectory.mkdir()
5757

5858
def apply(obj: Any): Unit = {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala

Lines changed: 2 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -19,20 +19,9 @@ package org.apache.spark.sql.catalyst
1919

2020
import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File}
2121

22-
import org.apache.spark.util.{Utils => SparkUtils}
22+
import org.apache.spark.util.Utils
2323

2424
package object util {
25-
/**
26-
* Returns a path to a temporary file that probably does not exist.
27-
* Note, there is always the race condition that someone created this
28-
* file since the last time we checked. Thus, this shouldn't be used
29-
* for anything security conscious.
30-
*/
31-
def getTempFilePath(prefix: String, suffix: String = ""): File = {
32-
val tempFile = File.createTempFile(prefix, suffix)
33-
tempFile.delete()
34-
tempFile
35-
}
3625

3726
def fileToString(file: File, encoding: String = "UTF-8") = {
3827
val inStream = new FileInputStream(file)
@@ -56,7 +45,7 @@ package object util {
5645
def resourceToString(
5746
resource:String,
5847
encoding: String = "UTF-8",
59-
classLoader: ClassLoader = SparkUtils.getSparkClassLoader) = {
48+
classLoader: ClassLoader = Utils.getSparkClassLoader) = {
6049
val inStream = classLoader.getResourceAsStream(resource)
6150
val outStream = new ByteArrayOutputStream
6251
try {

sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import scala.reflect.ClassTag
2323
import scala.reflect.runtime.universe.TypeTag
2424
import scala.util.Try
2525

26-
import org.apache.spark.sql.catalyst.util
2726
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
2827
import org.apache.spark.util.Utils
2928

@@ -67,8 +66,9 @@ private[sql] trait ParquetTest {
6766
* @todo Probably this method should be moved to a more general place
6867
*/
6968
protected def withTempPath(f: File => Unit): Unit = {
70-
val file = util.getTempFilePath("parquetTest").getCanonicalFile
71-
try f(file) finally if (file.exists()) Utils.deleteRecursively(file)
69+
val path = Utils.createTempDir()
70+
path.delete()
71+
try f(path) finally Utils.deleteRecursively(path)
7272
}
7373

7474
/**

sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@ import java.sql.{Date, Timestamp}
2222
import org.scalactic.Tolerance._
2323

2424
import org.apache.spark.sql.TestData._
25-
import org.apache.spark.sql.catalyst.util._
2625
import org.apache.spark.sql.functions._
2726
import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType}
2827
import org.apache.spark.sql.sources.LogicalRelation
@@ -31,6 +30,7 @@ import org.apache.spark.sql.test.TestSQLContext._
3130
import org.apache.spark.sql.test.TestSQLContext.implicits._
3231
import org.apache.spark.sql.types._
3332
import org.apache.spark.sql.{QueryTest, Row, SQLConf}
33+
import org.apache.spark.util.Utils
3434

3535
class JsonSuite extends QueryTest {
3636
import org.apache.spark.sql.json.TestJsonData._
@@ -554,8 +554,9 @@ class JsonSuite extends QueryTest {
554554
}
555555

556556
test("jsonFile should be based on JSONRelation") {
557-
val file = getTempFilePath("json")
558-
val path = file.toString
557+
val dir = Utils.createTempDir()
558+
dir.delete()
559+
val path = dir.getCanonicalPath
559560
sparkContext.parallelize(1 to 100).map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path)
560561
val jsonDF = jsonFile(path, 0.49)
561562

@@ -580,8 +581,9 @@ class JsonSuite extends QueryTest {
580581
}
581582

582583
test("Loading a JSON dataset from a text file") {
583-
val file = getTempFilePath("json")
584-
val path = file.toString
584+
val dir = Utils.createTempDir()
585+
dir.delete()
586+
val path = dir.getCanonicalPath
585587
primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
586588
val jsonDF = jsonFile(path)
587589

@@ -611,8 +613,9 @@ class JsonSuite extends QueryTest {
611613
}
612614

613615
test("Loading a JSON dataset from a text file with SQL") {
614-
val file = getTempFilePath("json")
615-
val path = file.toString
616+
val dir = Utils.createTempDir()
617+
dir.delete()
618+
val path = dir.getCanonicalPath
616619
primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
617620

618621
sql(
@@ -637,8 +640,9 @@ class JsonSuite extends QueryTest {
637640
}
638641

639642
test("Applying schemas") {
640-
val file = getTempFilePath("json")
641-
val path = file.toString
643+
val dir = Utils.createTempDir()
644+
dir.delete()
645+
val path = dir.getCanonicalPath
642646
primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
643647

644648
val schema = StructType(

sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@ import java.io.File
2222
import org.apache.spark.sql.AnalysisException
2323
import org.scalatest.BeforeAndAfterAll
2424

25-
import org.apache.spark.sql.catalyst.util
2625
import org.apache.spark.util.Utils
2726

2827
class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
@@ -32,7 +31,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
3231
var path: File = null
3332

3433
override def beforeAll(): Unit = {
35-
path = util.getTempFilePath("jsonCTAS").getCanonicalFile
34+
path = Utils.createTempDir()
3635
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
3736
jsonRDD(rdd).registerTempTable("jt")
3837
}
@@ -42,7 +41,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
4241
}
4342

4443
after {
45-
if (path.exists()) Utils.deleteRecursively(path)
44+
Utils.deleteRecursively(path)
4645
}
4746

4847
test("CREATE TEMPORARY TABLE AS SELECT") {

sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@ import java.io.File
2222
import org.scalatest.BeforeAndAfterAll
2323

2424
import org.apache.spark.sql.{AnalysisException, Row}
25-
import org.apache.spark.sql.catalyst.util
2625
import org.apache.spark.util.Utils
2726

2827
class InsertSuite extends DataSourceTest with BeforeAndAfterAll {
@@ -32,7 +31,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll {
3231
var path: File = null
3332

3433
override def beforeAll: Unit = {
35-
path = util.getTempFilePath("jsonCTAS").getCanonicalFile
34+
path = Utils.createTempDir()
3635
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
3736
jsonRDD(rdd).registerTempTable("jt")
3837
sql(
@@ -48,7 +47,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll {
4847
override def afterAll: Unit = {
4948
dropTempTable("jsonTable")
5049
dropTempTable("jt")
51-
if (path.exists()) Utils.deleteRecursively(path)
50+
Utils.deleteRecursively(path)
5251
}
5352

5453
test("Simple INSERT OVERWRITE a JSONRelation") {

sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@ import java.io.File
2121

2222
import org.scalatest.BeforeAndAfterAll
2323

24-
import org.apache.spark.sql.catalyst.util
2524
import org.apache.spark.sql.{SaveMode, SQLConf, DataFrame}
2625
import org.apache.spark.sql.types._
2726
import org.apache.spark.util.Utils
@@ -39,7 +38,8 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll {
3938
override def beforeAll(): Unit = {
4039
originalDefaultSource = conf.defaultDataSourceName
4140

42-
path = util.getTempFilePath("datasource").getCanonicalFile
41+
path = Utils.createTempDir()
42+
path.delete()
4343

4444
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
4545
df = jsonRDD(rdd)
@@ -52,7 +52,7 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll {
5252

5353
after {
5454
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource)
55-
if (path.exists()) Utils.deleteRecursively(path)
55+
Utils.deleteRecursively(path)
5656
}
5757

5858
def checkLoad(): Unit = {

sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars
2929
import org.scalatest.{BeforeAndAfterAll, FunSuite}
3030

3131
import org.apache.spark.Logging
32-
import org.apache.spark.sql.catalyst.util.getTempFilePath
32+
import org.apache.spark.util.Utils
3333

3434
class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
3535
def runCliWithin(
@@ -38,8 +38,8 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
3838
queriesAndExpectedAnswers: (String, String)*) {
3939

4040
val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip
41-
val warehousePath = getTempFilePath("warehouse")
42-
val metastorePath = getTempFilePath("metastore")
41+
val warehousePath = Utils.createTempDir()
42+
val metastorePath = Utils.createTempDir()
4343
val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator)
4444

4545
val command = {

sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,6 @@ import org.apache.thrift.transport.TSocket
3737
import org.scalatest.{BeforeAndAfterAll, FunSuite}
3838

3939
import org.apache.spark.Logging
40-
import org.apache.spark.sql.catalyst.util
4140
import org.apache.spark.sql.hive.HiveShim
4241
import org.apache.spark.util.Utils
4342

@@ -300,8 +299,8 @@ abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll wit
300299
}
301300

302301
private def startThriftServer(port: Int, attempt: Int) = {
303-
warehousePath = util.getTempFilePath("warehouse")
304-
metastorePath = util.getTempFilePath("metastore")
302+
warehousePath = Utils.createTempDir()
303+
metastorePath = Utils.createTempDir()
305304
logPath = null
306305
logTailingProcess = null
307306

0 commit comments

Comments
 (0)