aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorSean Owen <sowen@cloudera.com>2015-03-20 14:16:21 +0000
committerSean Owen <sowen@cloudera.com>2015-03-20 14:16:21 +0000
commit6f80c3e8880340597f161f87e64697bec86cc586 (patch)
treec019ca07ed3b4dd178c102aac00418485da5e679 /sql
parentd08e3eb3dc455970b685a7b8b7e00c537c89a8e4 (diff)
downloadspark-6f80c3e8880340597f161f87e64697bec86cc586.tar.gz
spark-6f80c3e8880340597f161f87e64697bec86cc586.tar.bz2
spark-6f80c3e8880340597f161f87e64697bec86cc586.zip
SPARK-6338 [CORE] Use standard temp dir mechanisms in tests to avoid orphaned temp files
Use `Utils.createTempDir()` to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify Author: Sean Owen <sowen@cloudera.com> Closes #5029 from srowen/SPARK-6338 and squashes the following commits: 27b740a [Sean Owen] Fix hive-thriftserver tests that don't expect an existing dir 4a212fa [Sean Owen] Standardize a bit more temp dir management 9004081 [Sean Owen] Revert some added recursive-delete calls 57609e4 [Sean Owen] Use Utils.createTempDir() to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala15
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala6
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala6
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala22
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala5
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala5
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala6
-rw-r--r--sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala8
-rw-r--r--sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala7
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala16
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala5
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala14
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala22
14 files changed, 61 insertions, 80 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala
index 80c7dfd376..528e38a50a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.catalyst.rules
-import org.apache.spark.sql.catalyst.util
+import org.apache.spark.util.Utils
/**
* A collection of generators that build custom bytecode at runtime for performing the evaluation
@@ -52,7 +52,7 @@ package object codegen {
@DeveloperApi
object DumpByteCode {
import scala.sys.process._
- val dumpDirectory = util.getTempFilePath("sparkSqlByteCode")
+ val dumpDirectory = Utils.createTempDir()
dumpDirectory.mkdir()
def apply(obj: Any): Unit = {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
index d8da45ae70..feed50f9a2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
@@ -19,20 +19,9 @@ package org.apache.spark.sql.catalyst
import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File}
-import org.apache.spark.util.{Utils => SparkUtils}
+import org.apache.spark.util.Utils
package object util {
- /**
- * Returns a path to a temporary file that probably does not exist.
- * Note, there is always the race condition that someone created this
- * file since the last time we checked. Thus, this shouldn't be used
- * for anything security conscious.
- */
- def getTempFilePath(prefix: String, suffix: String = ""): File = {
- val tempFile = File.createTempFile(prefix, suffix)
- tempFile.delete()
- tempFile
- }
def fileToString(file: File, encoding: String = "UTF-8") = {
val inStream = new FileInputStream(file)
@@ -56,7 +45,7 @@ package object util {
def resourceToString(
resource:String,
encoding: String = "UTF-8",
- classLoader: ClassLoader = SparkUtils.getSparkClassLoader) = {
+ classLoader: ClassLoader = Utils.getSparkClassLoader) = {
val inStream = classLoader.getResourceAsStream(resource)
val outStream = new ByteArrayOutputStream
try {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
index d6ea6679c5..9d17516e0e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
@@ -23,7 +23,6 @@ import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
import scala.util.Try
-import org.apache.spark.sql.catalyst.util
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
import org.apache.spark.util.Utils
@@ -67,8 +66,9 @@ private[sql] trait ParquetTest {
* @todo Probably this method should be moved to a more general place
*/
protected def withTempPath(f: File => Unit): Unit = {
- val file = util.getTempFilePath("parquetTest").getCanonicalFile
- try f(file) finally if (file.exists()) Utils.deleteRecursively(file)
+ val path = Utils.createTempDir()
+ path.delete()
+ try f(path) finally Utils.deleteRecursively(path)
}
/**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index 23f424c0bf..fe618e0e8e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -19,6 +19,8 @@ package org.apache.spark.sql
import java.io.File
+import org.apache.spark.util.Utils
+
import scala.beans.{BeanInfo, BeanProperty}
import org.apache.spark.rdd.RDD
@@ -98,13 +100,13 @@ class UserDefinedTypeSuite extends QueryTest {
test("UDTs with Parquet") {
- val tempDir = File.createTempFile("parquet", "test")
+ val tempDir = Utils.createTempDir()
tempDir.delete()
pointsRDD.saveAsParquetFile(tempDir.getCanonicalPath)
}
test("Repartition UDTs with Parquet") {
- val tempDir = File.createTempFile("parquet", "test")
+ val tempDir = Utils.createTempDir()
tempDir.delete()
pointsRDD.repartition(1).saveAsParquetFile(tempDir.getCanonicalPath)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index 320b80d80e..706c966ee0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -22,7 +22,6 @@ import java.sql.{Date, Timestamp}
import org.scalactic.Tolerance._
import org.apache.spark.sql.TestData._
-import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType}
import org.apache.spark.sql.sources.LogicalRelation
@@ -31,6 +30,7 @@ import org.apache.spark.sql.test.TestSQLContext._
import org.apache.spark.sql.test.TestSQLContext.implicits._
import org.apache.spark.sql.types._
import org.apache.spark.sql.{QueryTest, Row, SQLConf}
+import org.apache.spark.util.Utils
class JsonSuite extends QueryTest {
import org.apache.spark.sql.json.TestJsonData._
@@ -554,8 +554,9 @@ class JsonSuite extends QueryTest {
}
test("jsonFile should be based on JSONRelation") {
- val file = getTempFilePath("json")
- val path = file.toString
+ val dir = Utils.createTempDir()
+ dir.delete()
+ val path = dir.getCanonicalPath
sparkContext.parallelize(1 to 100).map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path)
val jsonDF = jsonFile(path, 0.49)
@@ -580,8 +581,9 @@ class JsonSuite extends QueryTest {
}
test("Loading a JSON dataset from a text file") {
- val file = getTempFilePath("json")
- val path = file.toString
+ val dir = Utils.createTempDir()
+ dir.delete()
+ val path = dir.getCanonicalPath
primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
val jsonDF = jsonFile(path)
@@ -611,8 +613,9 @@ class JsonSuite extends QueryTest {
}
test("Loading a JSON dataset from a text file with SQL") {
- val file = getTempFilePath("json")
- val path = file.toString
+ val dir = Utils.createTempDir()
+ dir.delete()
+ val path = dir.getCanonicalPath
primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
sql(
@@ -637,8 +640,9 @@ class JsonSuite extends QueryTest {
}
test("Applying schemas") {
- val file = getTempFilePath("json")
- val path = file.toString
+ val dir = Utils.createTempDir()
+ dir.delete()
+ val path = dir.getCanonicalPath
primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
val schema = StructType(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
index 60355414a4..2975a7fee4 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
@@ -22,7 +22,6 @@ import java.io.File
import org.apache.spark.sql.AnalysisException
import org.scalatest.BeforeAndAfterAll
-import org.apache.spark.sql.catalyst.util
import org.apache.spark.util.Utils
class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
@@ -32,7 +31,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
var path: File = null
override def beforeAll(): Unit = {
- path = util.getTempFilePath("jsonCTAS").getCanonicalFile
+ path = Utils.createTempDir()
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
jsonRDD(rdd).registerTempTable("jt")
}
@@ -42,7 +41,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
}
after {
- if (path.exists()) Utils.deleteRecursively(path)
+ Utils.deleteRecursively(path)
}
test("CREATE TEMPORARY TABLE AS SELECT") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
index b5b16f9546..80efe9728f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
@@ -22,7 +22,6 @@ import java.io.File
import org.scalatest.BeforeAndAfterAll
import org.apache.spark.sql.{AnalysisException, Row}
-import org.apache.spark.sql.catalyst.util
import org.apache.spark.util.Utils
class InsertSuite extends DataSourceTest with BeforeAndAfterAll {
@@ -32,7 +31,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll {
var path: File = null
override def beforeAll: Unit = {
- path = util.getTempFilePath("jsonCTAS").getCanonicalFile
+ path = Utils.createTempDir()
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
jsonRDD(rdd).registerTempTable("jt")
sql(
@@ -48,7 +47,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll {
override def afterAll: Unit = {
dropTempTable("jsonTable")
dropTempTable("jt")
- if (path.exists()) Utils.deleteRecursively(path)
+ Utils.deleteRecursively(path)
}
test("Simple INSERT OVERWRITE a JSONRelation") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala
index 607488ccfd..43bc8eb2d1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala
@@ -21,7 +21,6 @@ import java.io.File
import org.scalatest.BeforeAndAfterAll
-import org.apache.spark.sql.catalyst.util
import org.apache.spark.sql.{SaveMode, SQLConf, DataFrame}
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -39,7 +38,8 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll {
override def beforeAll(): Unit = {
originalDefaultSource = conf.defaultDataSourceName
- path = util.getTempFilePath("datasource").getCanonicalFile
+ path = Utils.createTempDir()
+ path.delete()
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
df = jsonRDD(rdd)
@@ -52,7 +52,7 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll {
after {
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource)
- if (path.exists()) Utils.deleteRecursively(path)
+ Utils.deleteRecursively(path)
}
def checkLoad(): Unit = {
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
index 8bca4b33b3..75738fa22b 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
@@ -29,7 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.scalatest.{BeforeAndAfterAll, FunSuite}
import org.apache.spark.Logging
-import org.apache.spark.sql.catalyst.util.getTempFilePath
+import org.apache.spark.util.Utils
class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
def runCliWithin(
@@ -38,8 +38,10 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
queriesAndExpectedAnswers: (String, String)*) {
val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip
- val warehousePath = getTempFilePath("warehouse")
- val metastorePath = getTempFilePath("metastore")
+ val warehousePath = Utils.createTempDir()
+ warehousePath.delete()
+ val metastorePath = Utils.createTempDir()
+ metastorePath.delete()
val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator)
val command = {
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index aff96e21a5..bf20acecb1 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -37,7 +37,6 @@ import org.apache.thrift.transport.TSocket
import org.scalatest.{BeforeAndAfterAll, FunSuite}
import org.apache.spark.Logging
-import org.apache.spark.sql.catalyst.util
import org.apache.spark.sql.hive.HiveShim
import org.apache.spark.util.Utils
@@ -447,8 +446,10 @@ abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll wit
}
private def startThriftServer(port: Int, attempt: Int) = {
- warehousePath = util.getTempFilePath("warehouse")
- metastorePath = util.getTempFilePath("metastore")
+ warehousePath = Utils.createTempDir()
+ warehousePath.delete()
+ metastorePath = Utils.createTempDir()
+ metastorePath.delete()
logPath = null
logTailingProcess = null
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 4859991e23..b4aee78046 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -30,7 +30,6 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerDe
import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.CacheTableCommand
import org.apache.spark.sql.hive._
import org.apache.spark.sql.hive.execution.HiveNativeCommand
@@ -69,22 +68,19 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
hiveconf.set("hive.plan.serialization.format", "javaXML")
- lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath
- lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath
+ lazy val warehousePath = Utils.createTempDir()
+ lazy val metastorePath = Utils.createTempDir()
/** Sets up the system initially or after a RESET command */
protected def configure(): Unit = {
+ warehousePath.delete()
+ metastorePath.delete()
setConf("javax.jdo.option.ConnectionURL",
s"jdbc:derby:;databaseName=$metastorePath;create=true")
- setConf("hive.metastore.warehouse.dir", warehousePath)
- Utils.registerShutdownDeleteDir(new File(warehousePath))
- Utils.registerShutdownDeleteDir(new File(metastorePath))
+ setConf("hive.metastore.warehouse.dir", warehousePath.toString)
}
- val testTempDir = File.createTempFile("testTempFiles", "spark.hive.tmp")
- testTempDir.delete()
- testTempDir.mkdir()
- Utils.registerShutdownDeleteDir(testTempDir)
+ val testTempDir = Utils.createTempDir()
// For some hive test case which contain ${system:test.tmp.dir}
System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
index d4b175fa44..381cd2a291 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
@@ -21,12 +21,11 @@ import java.io.File
import org.scalatest.BeforeAndAfter
-import com.google.common.io.Files
-
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.{QueryTest, _}
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
/* Implicits */
import org.apache.spark.sql.hive.test.TestHive._
@@ -112,7 +111,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter {
test("SPARK-4203:random partition directory order") {
sql("CREATE TABLE tmp_table (key int, value string)")
- val tmpDir = Files.createTempDir()
+ val tmpDir = Utils.createTempDir()
sql(s"CREATE TABLE table_with_partition(c1 string) PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) location '${tmpDir.toURI.toString}' ")
sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='1') SELECT 'blarr' FROM tmp_table")
sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='2') SELECT 'blarr' FROM tmp_table")
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 5d6a6f3b64..ff2e6ea9ea 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -19,13 +19,14 @@ package org.apache.spark.sql.hive
import java.io.File
+import scala.collection.mutable.ArrayBuffer
+
import org.scalatest.BeforeAndAfterEach
import org.apache.commons.io.FileUtils
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapred.InvalidInputException
-import org.apache.spark.sql.catalyst.util
import org.apache.spark.sql._
import org.apache.spark.util.Utils
import org.apache.spark.sql.types._
@@ -34,8 +35,6 @@ import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.parquet.ParquetRelation2
import org.apache.spark.sql.sources.LogicalRelation
-import scala.collection.mutable.ArrayBuffer
-
/**
* Tests for persisting tables created though the data sources API into the metastore.
*/
@@ -43,11 +42,12 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
override def afterEach(): Unit = {
reset()
- if (tempPath.exists()) Utils.deleteRecursively(tempPath)
+ Utils.deleteRecursively(tempPath)
}
val filePath = Utils.getSparkClassLoader.getResource("sample.json").getFile
- var tempPath: File = util.getTempFilePath("jsonCTAS").getCanonicalFile
+ var tempPath: File = Utils.createTempDir()
+ tempPath.delete()
test ("persistent JSON table") {
sql(
@@ -154,7 +154,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
}
test("check change without refresh") {
- val tempDir = File.createTempFile("sparksql", "json")
+ val tempDir = File.createTempFile("sparksql", "json", Utils.createTempDir())
tempDir.delete()
sparkContext.parallelize(("a", "b") :: Nil).toDF()
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
@@ -192,7 +192,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
}
test("drop, change, recreate") {
- val tempDir = File.createTempFile("sparksql", "json")
+ val tempDir = File.createTempFile("sparksql", "json", Utils.createTempDir())
tempDir.delete()
sparkContext.parallelize(("a", "b") :: Nil).toDF()
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
index 1904f5faef..d891c4e890 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation}
import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan}
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
// The data where the partitioning key exists only in the directory structure.
case class ParquetData(intField: Int, stringField: String)
@@ -579,13 +580,8 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll
var partitionedTableDirWithKeyAndComplexTypes: File = null
override def beforeAll(): Unit = {
- partitionedTableDir = File.createTempFile("parquettests", "sparksql")
- partitionedTableDir.delete()
- partitionedTableDir.mkdir()
-
- normalTableDir = File.createTempFile("parquettests", "sparksql")
- normalTableDir.delete()
- normalTableDir.mkdir()
+ partitionedTableDir = Utils.createTempDir()
+ normalTableDir = Utils.createTempDir()
(1 to 10).foreach { p =>
val partDir = new File(partitionedTableDir, s"p=$p")
@@ -601,9 +597,7 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll
.toDF()
.saveAsParquetFile(new File(normalTableDir, "normal").getCanonicalPath)
- partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql")
- partitionedTableDirWithKey.delete()
- partitionedTableDirWithKey.mkdir()
+ partitionedTableDirWithKey = Utils.createTempDir()
(1 to 10).foreach { p =>
val partDir = new File(partitionedTableDirWithKey, s"p=$p")
@@ -613,9 +607,7 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll
.saveAsParquetFile(partDir.getCanonicalPath)
}
- partitionedTableDirWithKeyAndComplexTypes = File.createTempFile("parquettests", "sparksql")
- partitionedTableDirWithKeyAndComplexTypes.delete()
- partitionedTableDirWithKeyAndComplexTypes.mkdir()
+ partitionedTableDirWithKeyAndComplexTypes = Utils.createTempDir()
(1 to 10).foreach { p =>
val partDir = new File(partitionedTableDirWithKeyAndComplexTypes, s"p=$p")
@@ -625,9 +617,7 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll
}.toDF().saveAsParquetFile(partDir.getCanonicalPath)
}
- partitionedTableDirWithComplexTypes = File.createTempFile("parquettests", "sparksql")
- partitionedTableDirWithComplexTypes.delete()
- partitionedTableDirWithComplexTypes.mkdir()
+ partitionedTableDirWithComplexTypes = Utils.createTempDir()
(1 to 10).foreach { p =>
val partDir = new File(partitionedTableDirWithComplexTypes, s"p=$p")