You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2017/03/20 13:36:08 UTC

spark git commit: [SPARK-19990][SQL][TEST-MAVEN] create a temp file for file in test.jar's resource when run mvn test accross different modules

Repository: spark
Updated Branches:
  refs/heads/master 816391159 -> 7ce30e00b


[SPARK-19990][SQL][TEST-MAVEN] create a temp file for file in test.jar's resource when run mvn test accross different modules

## What changes were proposed in this pull request?

After we have merged the `HiveDDLSuite` and `DDLSuite` in [SPARK-19235](https://issues.apache.org/jira/browse/SPARK-19235), we have two subclasses of `DDLSuite`, that is `HiveCatalogedDDLSuite` and `InMemoryCatalogDDLSuite`.

While `DDLSuite` is in `sql/core module`, and `HiveCatalogedDDLSuite` is in `sql/hive module`, if we mvn test
`HiveCatalogedDDLSuite`, it will run the test in its parent class `DDLSuite`, this will cause some test case failed which will get and use the test file path in `sql/core module` 's `resource`.

Because the test file path getted will start with 'jar:' like "jar:file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.6/sql/core/target/spark-sql_2.11-2.2.0-SNAPSHOT-tests.jar!/test-data/cars.csv", which will failed when new Path() in datasource.scala

This PR fix this by copy file from resource to  a temp dir.

## How was this patch tested?
N/A

Author: windpiger <so...@outlook.com>

Closes #17338 from windpiger/fixtestfailemvn.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7ce30e00
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7ce30e00
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7ce30e00

Branch: refs/heads/master
Commit: 7ce30e00b236e77b5175f797f9c6fc6cf4ca7e93
Parents: 8163911
Author: windpiger <so...@outlook.com>
Authored: Mon Mar 20 21:36:00 2017 +0800
Committer: Wenchen Fan <we...@databricks.com>
Committed: Mon Mar 20 21:36:00 2017 +0800

----------------------------------------------------------------------
 .../spark/sql/execution/command/DDLSuite.scala  | 33 ++++++++++++--------
 .../apache/spark/sql/test/SQLTestUtils.scala    | 17 +++++++++-
 2 files changed, 36 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/7ce30e00/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index dd76fdd..235c6bf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfterEach
 
 import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode}
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException}
+import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException}
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
 import org.apache.spark.sql.internal.SQLConf
@@ -699,21 +699,28 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
   }
 
   test("create temporary view using") {
-    val csvFile =
-      Thread.currentThread().getContextClassLoader.getResource("test-data/cars.csv").toString
-    withView("testview") {
-      sql(s"CREATE OR REPLACE TEMPORARY VIEW testview (c1 String, c2 String)  USING " +
-        "org.apache.spark.sql.execution.datasources.csv.CSVFileFormat  " +
-        s"OPTIONS (PATH '$csvFile')")
+    // when we test the HiveCatalogedDDLSuite, it will failed because the csvFile path above
+    // starts with 'jar:', and it is an illegal parameter for Path, so here we copy it
+    // to a temp file by withResourceTempPath
+    withResourceTempPath("test-data/cars.csv") { tmpFile =>
+      withView("testview") {
+        sql(s"CREATE OR REPLACE TEMPORARY VIEW testview (c1 String, c2 String)  USING " +
+          "org.apache.spark.sql.execution.datasources.csv.CSVFileFormat  " +
+          s"OPTIONS (PATH '$tmpFile')")
 
-      checkAnswer(
-        sql("select c1, c2 from testview order by c1 limit 1"),
+        checkAnswer(
+          sql("select c1, c2 from testview order by c1 limit 1"),
           Row("1997", "Ford") :: Nil)
 
-      // Fails if creating a new view with the same name
-      intercept[TempTableAlreadyExistsException] {
-        sql(s"CREATE TEMPORARY VIEW testview USING " +
-          s"org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '$csvFile')")
+        // Fails if creating a new view with the same name
+        intercept[TempTableAlreadyExistsException] {
+          sql(
+            s"""
+               |CREATE TEMPORARY VIEW testview
+               |USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat
+               |OPTIONS (PATH '$tmpFile')
+             """.stripMargin)
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/7ce30e00/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
index 9201954..cab2192 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
@@ -19,10 +19,10 @@ package org.apache.spark.sql.test
 
 import java.io.File
 import java.net.URI
+import java.nio.file.Files
 import java.util.UUID
 
 import scala.language.implicitConversions
-import scala.util.Try
 import scala.util.control.NonFatal
 
 import org.apache.hadoop.fs.Path
@@ -124,6 +124,21 @@ private[sql] trait SQLTestUtils
   }
 
   /**
+   * Copy file in jar's resource to a temp file, then pass it to `f`.
+   * This function is used to make `f` can use the path of temp file(e.g. file:/), instead of
+   * path of jar's resource which starts with 'jar:file:/'
+   */
+  protected def withResourceTempPath(resourcePath: String)(f: File => Unit): Unit = {
+    val inputStream =
+      Thread.currentThread().getContextClassLoader.getResourceAsStream(resourcePath)
+    withTempDir { dir =>
+      val tmpFile = new File(dir, "tmp")
+      Files.copy(inputStream, tmpFile.toPath)
+      f(tmpFile)
+    }
+  }
+
+  /**
    * Creates a temporary directory, which is then passed to `f` and will be deleted after `f`
    * returns.
    *


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org