Repository: spark
Updated Branches:
refs/heads/branch-2.2 022a4957d -> 5ae1c6521
[SPARK-19707][SPARK-18922][TESTS][SQL][CORE] Fix test failures/the invalid path
check for sc.addJar on Windows
## What changes were proposed in this pull request?
This PR proposes two things:
- A follow up for SPARK-19707 (Improving the invalid path check for sc.addJar
on Windows as well).
```
org.apache.spark.SparkContextSuite:
- add jar with invalid path *** FAILED *** (32 milliseconds)
2 was not equal to 1 (SparkContextSuite.scala:309)
...
```
- Fix path vs URI related test failures on Windows.
```
org.apache.spark.storage.LocalDirsSuite:
- SPARK_LOCAL_DIRS override also affects driver *** FAILED *** (0 milliseconds)
new java.io.File("/NONEXISTENT_PATH").exists() was true
(LocalDirsSuite.scala:50)
...
- Utils.getLocalDir() throws an exception if any temporary directory cannot be
retrieved *** FAILED *** (15 milliseconds)
Expected exception java.io.IOException to be thrown, but no exception was
thrown. (LocalDirsSuite.scala:64)
...
```
```
org.apache.spark.sql.hive.HiveSchemaInferenceSuite:
- orc: schema should be inferred and saved when INFER_AND_SAVE is specified
*** FAILED *** (203 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2:
C:\projects\spark\target\tmp\spark-dae61ab3-a851-4dd3-bf4e-be97c501f254
...
- parquet: schema should be inferred and saved when INFER_AND_SAVE is
specified *** FAILED *** (203 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2:
C:\projects\spark\target\tmp\spark-fa3aff89-a66e-4376-9a37-2a9b87596939
...
- orc: schema should be inferred but not stored when INFER_ONLY is specified
*** FAILED *** (141 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2:
C:\projects\spark\target\tmp\spark-fb464e59-b049-481b-9c75-f53295c9fc2c
...
- parquet: schema should be inferred but not stored when INFER_ONLY is
specified *** FAILED *** (125 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2:
C:\projects\spark\target\tmp\spark-9487568e-80a4-42b3-b0a5-d95314c4ccbc
...
- orc: schema should not be inferred when NEVER_INFER is specified *** FAILED
*** (156 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2:
C:\projects\spark\target\tmp\spark-0d2dfa45-1b0f-4958-a8be-1074ed0135a
...
- parquet: schema should not be inferred when NEVER_INFER is specified ***
FAILED *** (547 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2:
C:\projects\spark\target\tmp\spark-6d95d64e-613e-4a59-a0f6-d198c5aa51ee
...
```
```
org.apache.spark.sql.execution.command.DDLSuite:
- create temporary view using *** FAILED *** (15 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist:
file:/C:projectsspark arget mpspark-3881d9ca-561b-488d-90b9-97587472b853
mp;
...
- insert data to a data source table which has a non-existing location should
succeed *** FAILED *** (109 milliseconds)
file:/C:projectsspark%09arget%09mpspark-4cad3d19-6085-4b75-b407-fe5e9d21df54
did not equal
file:///C:/projects/spark/target/tmp/spark-4cad3d19-6085-4b75-b407-fe5e9d21df54
(DDLSuite.scala:1869)
...
- insert into a data source table with a non-existing partition location
should succeed *** FAILED *** (94 milliseconds)
file:/C:projectsspark%09arget%09mpspark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d
did not equal
file:///C:/projects/spark/target/tmp/spark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d
(DDLSuite.scala:1910)
...
- read data from a data source table which has a non-existing location should
succeed *** FAILED *** (93 milliseconds)
file:/C:projectsspark%09arget%09mpspark-f8c281e2-08c2-4f73-abbf-f3865b702c34
did not equal
file:///C:/projects/spark/target/tmp/spark-f8c281e2-08c2-4f73-abbf-f3865b702c34
(DDLSuite.scala:1937)
...
- read data from a data source table with non-existing partition location
should succeed *** FAILED *** (110 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- create datasource table with a non-existing location *** FAILED *** (94
milliseconds)
file:/C:projectsspark%09arget%09mpspark-387316ae-070c-4e78-9b78-19ebf7b29ec8
did not equal
file:///C:/projects/spark/target/tmp/spark-387316ae-070c-4e78-9b78-19ebf7b29ec8
(DDLSuite.scala:1982)
...
- CTAS for external data source table with a non-existing location *** FAILED
*** (16 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- CTAS for external data source table with a existed location *** FAILED ***
(15 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- data source table:partition column name containing a b *** FAILED *** (125
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- data source table:partition column name containing a:b *** FAILED *** (143
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- data source table:partition column name containing a%b *** FAILED *** (109
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- data source table:partition column name containing a,b *** FAILED *** (109
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- location uri contains a b for datasource table *** FAILED *** (94
milliseconds)
file:/C:projectsspark%09arget%09mpspark-5739cda9-b702-4e14-932c-42e8c4174480a%20b
did not equal
file:///C:/projects/spark/target/tmp/spark-5739cda9-b702-4e14-932c-42e8c4174480/a%20b
(DDLSuite.scala:2084)
...
- location uri contains a:b for datasource table *** FAILED *** (78
milliseconds)
file:/C:projectsspark%09arget%09mpspark-9bdd227c-840f-4f08-b7c5-4036638f098da:b
did not equal
file:///C:/projects/spark/target/tmp/spark-9bdd227c-840f-4f08-b7c5-4036638f098d/a:b
(DDLSuite.scala:2084)
...
- location uri contains a%b for datasource table *** FAILED *** (78
milliseconds)
file:/C:projectsspark%09arget%09mpspark-62bb5f1d-fa20-460a-b534-cb2e172a3640a%25b
did not equal
file:///C:/projects/spark/target/tmp/spark-62bb5f1d-fa20-460a-b534-cb2e172a3640/a%25b
(DDLSuite.scala:2084)
...
- location uri contains a b for database *** FAILED *** (16 milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- location uri contains a:b for database *** FAILED *** (15 milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- location uri contains a%b for database *** FAILED *** (0 milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
```
```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
- create hive table with a non-existing location *** FAILED *** (16
milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- CTAS for external hive table with a non-existing location *** FAILED *** (16
milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- CTAS for external hive table with a existed location *** FAILED *** (16
milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- partition column name of parquet table containing a b *** FAILED *** (156
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- partition column name of parquet table containing a:b *** FAILED *** (94
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- partition column name of parquet table containing a%b *** FAILED *** (125
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- partition column name of parquet table containing a,b *** FAILED *** (110
milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
- partition column name of hive table containing a b *** FAILED *** (15
milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- partition column name of hive table containing a:b *** FAILED *** (16
milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- partition column name of hive table containing a%b *** FAILED *** (16
milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- partition column name of hive table containing a,b *** FAILED *** (0
milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- hive table: location uri contains a b *** FAILED *** (0 milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- hive table: location uri contains a:b *** FAILED *** (0 milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
- hive table: location uri contains a%b *** FAILED *** (0 milliseconds)
org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.HiveException:
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path
from an empty string);
...
```
```
org.apache.spark.sql.sources.PathOptionSuite:
- path option also exist for write path *** FAILED *** (94 milliseconds)
file:/C:projectsspark%09arget%09mpspark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc
did not equal
file:///C:/projects/spark/target/tmp/spark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc
(PathOptionSuite.scala:98)
...
```
```
org.apache.spark.sql.CachedTableSuite:
- SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to
this table *** FAILED *** (110 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty
string
...
```
```
org.apache.spark.sql.execution.DataSourceScanExecRedactionSuite:
- treeString is redacted *** FAILED *** (250 milliseconds)
"file:/C:/projects/spark/target/tmp/spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28"
did not contain
"C:\projects\spark\target\tmp\spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28"
(DataSourceScanExecRedactionSuite.scala:46)
...
```
## How was this patch tested?
Tested via AppVeyor for each and checked it passed once each. These should be
retested via AppVeyor in this PR.
Author: hyukjinkwon <[email protected]>
Closes #17987 from HyukjinKwon/windows-20170515.
(cherry picked from commit e9f983df275c138626af35fd263a7abedf69297f)
Signed-off-by: Sean Owen <[email protected]>
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5ae1c652
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5ae1c652
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5ae1c652
Branch: refs/heads/branch-2.2
Commit: 5ae1c652147aba9c5087335b0c6916a1035090b2
Parents: 022a495
Author: hyukjinkwon <[email protected]>
Authored: Thu May 25 17:10:30 2017 +0100
Committer: Sean Owen <[email protected]>
Committed: Thu May 25 17:10:38 2017 +0100
----------------------------------------------------------------------
.../scala/org/apache/spark/SparkContext.scala | 47 ++++++------
.../org/apache/spark/SparkContextSuite.scala | 6 +-
.../apache/spark/storage/LocalDirsSuite.scala | 33 +++++++-
.../org/apache/spark/sql/CachedTableSuite.scala | 2 +-
.../DataSourceScanExecRedactionSuite.scala | 2 +-
.../spark/sql/execution/command/DDLSuite.scala | 81 ++++++++++++++------
.../spark/sql/sources/PathOptionSuite.scala | 4 +-
.../sql/hive/HiveSchemaInferenceSuite.scala | 2 +-
.../spark/sql/hive/execution/HiveDDLSuite.scala | 41 ++++++----
9 files changed, 145 insertions(+), 73 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 7dbceb9..1a2443f 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1801,40 +1801,39 @@ class SparkContext(config: SparkConf) extends Logging {
* an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node.
*/
def addJar(path: String) {
+ def addJarFile(file: File): String = {
+ try {
+ if (!file.exists()) {
+ throw new FileNotFoundException(s"Jar ${file.getAbsolutePath} not
found")
+ }
+ if (file.isDirectory) {
+ throw new IllegalArgumentException(
+ s"Directory ${file.getAbsoluteFile} is not allowed for addJar")
+ }
+ env.rpcEnv.fileServer.addJar(file)
+ } catch {
+ case NonFatal(e) =>
+ logError(s"Failed to add $path to Spark environment", e)
+ null
+ }
+ }
+
if (path == null) {
logWarning("null specified as parameter to addJar")
} else {
- var key = ""
- if (path.contains("\\")) {
+ val key = if (path.contains("\\")) {
// For local paths with backslashes on Windows, URI throws an exception
- key = env.rpcEnv.fileServer.addJar(new File(path))
+ addJarFile(new File(path))
} else {
val uri = new URI(path)
// SPARK-17650: Make sure this is a valid URL before adding it to the
list of dependencies
Utils.validateURL(uri)
- key = uri.getScheme match {
+ uri.getScheme match {
// A JAR file which exists only on the driver node
- case null | "file" =>
- try {
- val file = new File(uri.getPath)
- if (!file.exists()) {
- throw new FileNotFoundException(s"Jar ${file.getAbsolutePath}
not found")
- }
- if (file.isDirectory) {
- throw new IllegalArgumentException(
- s"Directory ${file.getAbsoluteFile} is not allowed for
addJar")
- }
- env.rpcEnv.fileServer.addJar(new File(uri.getPath))
- } catch {
- case NonFatal(e) =>
- logError(s"Failed to add $path to Spark environment", e)
- null
- }
+ case null | "file" => addJarFile(new File(uri.getPath))
// A JAR file which exists locally on every worker node
- case "local" =>
- "file:" + uri.getPath
- case _ =>
- path
+ case "local" => "file:" + uri.getPath
+ case _ => path
}
}
if (key != null) {
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index 27945a9..979270a 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -300,13 +300,13 @@ class SparkContextSuite extends SparkFunSuite with
LocalSparkContext with Eventu
sc = new SparkContext(new
SparkConf().setAppName("test").setMaster("local"))
sc.addJar(tmpJar.getAbsolutePath)
- // Invaid jar path will only print the error log, will not add to file
server.
+ // Invalid jar path will only print the error log, will not add to file
server.
sc.addJar("dummy.jar")
sc.addJar("")
sc.addJar(tmpDir.getAbsolutePath)
- sc.listJars().size should be (1)
- sc.listJars().head should include (tmpJar.getName)
+ assert(sc.listJars().size == 1)
+ assert(sc.listJars().head.contains(tmpJar.getName))
}
test("Cancelling job group should not cause SparkContext to shutdown
(SPARK-6414)") {
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
index f7b3a27..6883eb2 100644
--- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
@@ -37,27 +37,50 @@ class LocalDirsSuite extends SparkFunSuite with
BeforeAndAfter {
Utils.clearLocalRootDirs()
}
+ private def assumeNonExistentAndNotCreatable(f: File): Unit = {
+ try {
+ assume(!f.exists() && !f.mkdirs())
+ } finally {
+ Utils.deleteRecursively(f)
+ }
+ }
+
test("Utils.getLocalDir() returns a valid directory, even if some local dirs
are missing") {
// Regression test for SPARK-2974
- assert(!new File("/NONEXISTENT_PATH").exists())
+ val f = new File("/NONEXISTENT_PATH")
+ assumeNonExistentAndNotCreatable(f)
+
val conf = new SparkConf(false)
.set("spark.local.dir",
s"/NONEXISTENT_PATH,${System.getProperty("java.io.tmpdir")}")
assert(new File(Utils.getLocalDir(conf)).exists())
+
+ // This directory should not be created.
+ assert(!f.exists())
}
test("SPARK_LOCAL_DIRS override also affects driver") {
- // Regression test for SPARK-2975
- assert(!new File("/NONEXISTENT_PATH").exists())
+ // Regression test for SPARK-2974
+ val f = new File("/NONEXISTENT_PATH")
+ assumeNonExistentAndNotCreatable(f)
+
// spark.local.dir only contains invalid directories, but that's not a
problem since
// SPARK_LOCAL_DIRS will override it on both the driver and workers:
val conf = new SparkConfWithEnv(Map("SPARK_LOCAL_DIRS" ->
System.getProperty("java.io.tmpdir")))
.set("spark.local.dir", "/NONEXISTENT_PATH")
assert(new File(Utils.getLocalDir(conf)).exists())
+
+ // This directory should not be created.
+ assert(!f.exists())
}
test("Utils.getLocalDir() throws an exception if any temporary directory
cannot be retrieved") {
val path1 = "/NONEXISTENT_PATH_ONE"
val path2 = "/NONEXISTENT_PATH_TWO"
+ val f1 = new File(path1)
+ val f2 = new File(path2)
+ assumeNonExistentAndNotCreatable(f1)
+ assumeNonExistentAndNotCreatable(f2)
+
assert(!new File(path1).exists())
assert(!new File(path2).exists())
val conf = new SparkConf(false).set("spark.local.dir", s"$path1,$path2")
@@ -67,5 +90,9 @@ class LocalDirsSuite extends SparkFunSuite with
BeforeAndAfter {
// If any temporary directory could not be retrieved under the given paths
above, it should
// throw an exception with the message that includes the paths.
assert(message.contains(s"$path1,$path2"))
+
+ // These directories should not be created.
+ assert(!f1.exists())
+ assert(!f2.exists())
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
----------------------------------------------------------------------
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
index e66fe97..3ad5268 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
@@ -647,7 +647,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils
with SharedSQLContext
withTable("t") {
withTempPath { path =>
Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath)
- sql(s"CREATE TABLE t USING parquet LOCATION '$path'")
+ sql(s"CREATE TABLE t USING parquet LOCATION '${path.toURI}'")
spark.catalog.cacheTable("t")
spark.table("t").select($"i").cache()
checkAnswer(spark.table("t").select($"i"), Row(1))
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
----------------------------------------------------------------------
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
index f7f1cce..423e128 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
@@ -38,7 +38,7 @@ class DataSourceScanExecRedactionSuite extends QueryTest with
SharedSQLContext {
val rootPath =
df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get
.asInstanceOf[FileSourceScanExec].relation.location.rootPaths.head
- assert(rootPath.toString.contains(basePath.toString))
+ assert(rootPath.toString.contains(dir.toURI.getPath.stripSuffix("/")))
assert(!df.queryExecution.sparkPlan.treeString(verbose =
true).contains(rootPath.getName))
assert(!df.queryExecution.executedPlan.treeString(verbose =
true).contains(rootPath.getName))
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/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 0abcff7..e4dd077 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
@@ -702,7 +702,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils
{
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')")
+ s"OPTIONS (PATH '${tmpFile.toURI}')")
checkAnswer(
sql("select c1, c2 from testview order by c1 limit 1"),
@@ -714,7 +714,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils
{
s"""
|CREATE TEMPORARY VIEW testview
|USING
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat
- |OPTIONS (PATH '$tmpFile')
+ |OPTIONS (PATH '${tmpFile.toURI}')
""".stripMargin)
}
}
@@ -1835,7 +1835,7 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
s"""
|CREATE TABLE t(a string, b int)
|USING parquet
- |OPTIONS(path "$dir")
+ |OPTIONS(path "${dir.toURI}")
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.location == makeQualifiedPath(dir.getAbsolutePath))
@@ -1853,12 +1853,12 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
checkAnswer(spark.table("t"), Row("c", 1) :: Nil)
val newDirFile = new File(dir, "x")
- val newDir = newDirFile.getAbsolutePath
+ val newDir = newDirFile.toURI
spark.sql(s"ALTER TABLE t SET LOCATION '$newDir'")
spark.sessionState.catalog.refreshTable(TableIdentifier("t"))
val table1 =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
- assert(table1.location == new URI(newDir))
+ assert(table1.location == newDir)
assert(!newDirFile.exists)
spark.sql("INSERT INTO TABLE t SELECT 'c', 1")
@@ -1876,7 +1876,7 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
|CREATE TABLE t(a int, b int, c int, d int)
|USING parquet
|PARTITIONED BY(a, b)
- |LOCATION "$dir"
+ |LOCATION "${dir.toURI}"
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.location == makeQualifiedPath(dir.getAbsolutePath))
@@ -1902,7 +1902,7 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
s"""
|CREATE TABLE t(a string, b int)
|USING parquet
- |OPTIONS(path "$dir")
+ |OPTIONS(path "${dir.toURI}")
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
@@ -1931,7 +1931,7 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
|CREATE TABLE t(a int, b int, c int, d int)
|USING parquet
|PARTITIONED BY(a, b)
- |LOCATION "$dir"
+ |LOCATION "${dir.toURI}"
""".stripMargin)
spark.sql("INSERT INTO TABLE t PARTITION(a=1, b=2) SELECT 3, 4")
checkAnswer(spark.table("t"), Row(3, 4, 1, 2) :: Nil)
@@ -1948,7 +1948,7 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
test("create datasource table with a non-existing location") {
withTable("t", "t1") {
withTempPath { dir =>
- spark.sql(s"CREATE TABLE t(a int, b int) USING parquet LOCATION
'$dir'")
+ spark.sql(s"CREATE TABLE t(a int, b int) USING parquet LOCATION
'${dir.toURI}'")
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.location == makeQualifiedPath(dir.getAbsolutePath))
@@ -1960,7 +1960,8 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
}
// partition table
withTempPath { dir =>
- spark.sql(s"CREATE TABLE t1(a int, b int) USING parquet PARTITIONED
BY(a) LOCATION '$dir'")
+ spark.sql(
+ s"CREATE TABLE t1(a int, b int) USING parquet PARTITIONED BY(a)
LOCATION '${dir.toURI}'")
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
assert(table.location == makeQualifiedPath(dir.getAbsolutePath))
@@ -1985,7 +1986,7 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
s"""
|CREATE TABLE t
|USING parquet
- |LOCATION '$dir'
+ |LOCATION '${dir.toURI}'
|AS SELECT 3 as a, 4 as b, 1 as c, 2 as d
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
@@ -2001,7 +2002,7 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
|CREATE TABLE t1
|USING parquet
|PARTITIONED BY(a, b)
- |LOCATION '$dir'
+ |LOCATION '${dir.toURI}'
|AS SELECT 3 as a, 4 as b, 1 as c, 2 as d
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
@@ -2018,6 +2019,10 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
Seq("a b", "a:b", "a%b", "a,b").foreach { specialChars =>
test(s"data source table:partition column name containing $specialChars") {
+ // On Windows, it looks colon in the file name is illegal by default. See
+ // https://support.microsoft.com/en-us/help/289627
+ assume(!Utils.isWindows || specialChars != "a:b")
+
withTable("t") {
withTempDir { dir =>
spark.sql(
@@ -2025,14 +2030,14 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
|CREATE TABLE t(a string, `$specialChars` string)
|USING parquet
|PARTITIONED BY(`$specialChars`)
- |LOCATION '$dir'
+ |LOCATION '${dir.toURI}'
""".stripMargin)
assert(dir.listFiles().isEmpty)
spark.sql(s"INSERT INTO TABLE t PARTITION(`$specialChars`=2) SELECT
1")
val partEscaped =
s"${ExternalCatalogUtils.escapePathName(specialChars)}=2"
val partFile = new File(dir, partEscaped)
- assert(partFile.listFiles().length >= 1)
+ assert(partFile.listFiles().nonEmpty)
checkAnswer(spark.table("t"), Row("1", "2") :: Nil)
}
}
@@ -2041,15 +2046,22 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
Seq("a b", "a:b", "a%b").foreach { specialChars =>
test(s"location uri contains $specialChars for datasource table") {
+ // On Windows, it looks colon in the file name is illegal by default. See
+ // https://support.microsoft.com/en-us/help/289627
+ assume(!Utils.isWindows || specialChars != "a:b")
+
withTable("t", "t1") {
withTempDir { dir =>
val loc = new File(dir, specialChars)
loc.mkdir()
+ // The parser does not recognize the backslashes on Windows as they
are.
+ // These currently should be escaped.
+ val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\")
spark.sql(
s"""
|CREATE TABLE t(a string)
|USING parquet
- |LOCATION '$loc'
+ |LOCATION '$escapedLoc'
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
@@ -2058,19 +2070,22 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
assert(loc.listFiles().isEmpty)
spark.sql("INSERT INTO TABLE t SELECT 1")
- assert(loc.listFiles().length >= 1)
+ assert(loc.listFiles().nonEmpty)
checkAnswer(spark.table("t"), Row("1") :: Nil)
}
withTempDir { dir =>
val loc = new File(dir, specialChars)
loc.mkdir()
+ // The parser does not recognize the backslashes on Windows as they
are.
+ // These currently should be escaped.
+ val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\")
spark.sql(
s"""
|CREATE TABLE t1(a string, b string)
|USING parquet
|PARTITIONED BY(b)
- |LOCATION '$loc'
+ |LOCATION '$escapedLoc'
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
@@ -2080,15 +2095,20 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
assert(loc.listFiles().isEmpty)
spark.sql("INSERT INTO TABLE t1 PARTITION(b=2) SELECT 1")
val partFile = new File(loc, "b=2")
- assert(partFile.listFiles().length >= 1)
+ assert(partFile.listFiles().nonEmpty)
checkAnswer(spark.table("t1"), Row("1", "2") :: Nil)
spark.sql("INSERT INTO TABLE t1 PARTITION(b='2017-03-03 12:13%3A14')
SELECT 1")
val partFile1 = new File(loc, "b=2017-03-03 12:13%3A14")
assert(!partFile1.exists())
- val partFile2 = new File(loc, "b=2017-03-03 12%3A13%253A14")
- assert(partFile2.listFiles().length >= 1)
- checkAnswer(spark.table("t1"), Row("1", "2") :: Row("1", "2017-03-03
12:13%3A14") :: Nil)
+
+ if (!Utils.isWindows) {
+ // Actual path becomes "b=2017-03-03%2012%3A13%253A14" on Windows.
+ val partFile2 = new File(loc, "b=2017-03-03 12%3A13%253A14")
+ assert(partFile2.listFiles().nonEmpty)
+ checkAnswer(
+ spark.table("t1"), Row("1", "2") :: Row("1", "2017-03-03
12:13%3A14") :: Nil)
+ }
}
}
}
@@ -2096,11 +2116,18 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
Seq("a b", "a:b", "a%b").foreach { specialChars =>
test(s"location uri contains $specialChars for database") {
+ // On Windows, it looks colon in the file name is illegal by default. See
+ // https://support.microsoft.com/en-us/help/289627
+ assume(!Utils.isWindows || specialChars != "a:b")
+
withDatabase ("tmpdb") {
withTable("t") {
withTempDir { dir =>
val loc = new File(dir, specialChars)
- spark.sql(s"CREATE DATABASE tmpdb LOCATION '$loc'")
+ // The parser does not recognize the backslashes on Windows as
they are.
+ // These currently should be escaped.
+ val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\")
+ spark.sql(s"CREATE DATABASE tmpdb LOCATION '$escapedLoc'")
spark.sql("USE tmpdb")
import testImplicits._
@@ -2119,11 +2146,14 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
withTable("t", "t1") {
withTempDir { dir =>
assert(!dir.getAbsolutePath.startsWith("file:/"))
+ // The parser does not recognize the backslashes on Windows as they
are.
+ // These currently should be escaped.
+ val escapedDir = dir.getAbsolutePath.replace("\\", "\\\\")
spark.sql(
s"""
|CREATE TABLE t(a string)
|USING parquet
- |LOCATION '$dir'
+ |LOCATION '$escapedDir'
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.location.toString.startsWith("file:/"))
@@ -2131,12 +2161,15 @@ abstract class DDLSuite extends QueryTest with
SQLTestUtils {
withTempDir { dir =>
assert(!dir.getAbsolutePath.startsWith("file:/"))
+ // The parser does not recognize the backslashes on Windows as they
are.
+ // These currently should be escaped.
+ val escapedDir = dir.getAbsolutePath.replace("\\", "\\\\")
spark.sql(
s"""
|CREATE TABLE t1(a string, b string)
|USING parquet
|PARTITIONED BY(b)
- |LOCATION '$dir'
+ |LOCATION '$escapedDir'
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
assert(table.location.toString.startsWith("file:/"))
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
----------------------------------------------------------------------
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
index 6dd4847..c25c3f6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
@@ -92,12 +92,12 @@ class PathOptionSuite extends DataSourceTest with
SharedSQLContext {
s"""
|CREATE TABLE src
|USING ${classOf[TestOptionsSource].getCanonicalName}
- |OPTIONS (PATH '$p')
+ |OPTIONS (PATH '${p.toURI}')
|AS SELECT 1
""".stripMargin)
assert(
spark.table("src").schema.head.metadata.getString("path") ==
- p.getAbsolutePath)
+ p.toURI.toString)
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
----------------------------------------------------------------------
diff --git
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
index 319d026..b3a0604 100644
---
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
+++
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
@@ -104,7 +104,7 @@ class HiveSchemaInferenceSuite
identifier = TableIdentifier(table = TEST_TABLE_NAME, database =
Option(DATABASE)),
tableType = CatalogTableType.EXTERNAL,
storage = CatalogStorageFormat(
- locationUri = Option(new java.net.URI(dir.getAbsolutePath)),
+ locationUri = Option(dir.toURI),
inputFormat = serde.inputFormat,
outputFormat = serde.outputFormat,
serde = serde.serde,
http://git-wip-us.apache.org/repos/asf/spark/blob/5ae1c652/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
----------------------------------------------------------------------
diff --git
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index c3d734e..b282acf 100644
---
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -36,6 +36,7 @@ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
// TODO(gatorsmile): combine HiveCatalogedDDLSuite and HiveDDLSuite
class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with
BeforeAndAfterEach {
@@ -1636,7 +1637,7 @@ class HiveDDLSuite
test("create hive table with a non-existing location") {
withTable("t", "t1") {
withTempPath { dir =>
- spark.sql(s"CREATE TABLE t(a int, b int) USING hive LOCATION '$dir'")
+ spark.sql(s"CREATE TABLE t(a int, b int) USING hive LOCATION
'${dir.toURI}'")
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.location == makeQualifiedPath(dir.getAbsolutePath))
@@ -1653,7 +1654,7 @@ class HiveDDLSuite
|CREATE TABLE t1(a int, b int)
|USING hive
|PARTITIONED BY(a)
- |LOCATION '$dir'
+ |LOCATION '${dir.toURI}'
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
@@ -1681,7 +1682,7 @@ class HiveDDLSuite
s"""
|CREATE TABLE t
|USING hive
- |LOCATION '$dir'
+ |LOCATION '${dir.toURI}'
|AS SELECT 3 as a, 4 as b, 1 as c, 2 as d
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
@@ -1697,7 +1698,7 @@ class HiveDDLSuite
|CREATE TABLE t1
|USING hive
|PARTITIONED BY(a, b)
- |LOCATION '$dir'
+ |LOCATION '${dir.toURI}'
|AS SELECT 3 as a, 4 as b, 1 as c, 2 as d
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
@@ -1723,21 +1724,21 @@ class HiveDDLSuite
|CREATE TABLE t(a string, `$specialChars` string)
|USING $datasource
|PARTITIONED BY(`$specialChars`)
- |LOCATION '$dir'
+ |LOCATION '${dir.toURI}'
""".stripMargin)
assert(dir.listFiles().isEmpty)
spark.sql(s"INSERT INTO TABLE t PARTITION(`$specialChars`=2)
SELECT 1")
val partEscaped =
s"${ExternalCatalogUtils.escapePathName(specialChars)}=2"
val partFile = new File(dir, partEscaped)
- assert(partFile.listFiles().length >= 1)
+ assert(partFile.listFiles().nonEmpty)
checkAnswer(spark.table("t"), Row("1", "2") :: Nil)
withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") {
spark.sql(s"INSERT INTO TABLE t PARTITION(`$specialChars`)
SELECT 3, 4")
val partEscaped1 =
s"${ExternalCatalogUtils.escapePathName(specialChars)}=4"
val partFile1 = new File(dir, partEscaped1)
- assert(partFile1.listFiles().length >= 1)
+ assert(partFile1.listFiles().nonEmpty)
checkAnswer(spark.table("t"), Row("1", "2") :: Row("3", "4") ::
Nil)
}
}
@@ -1748,15 +1749,22 @@ class HiveDDLSuite
Seq("a b", "a:b", "a%b").foreach { specialChars =>
test(s"hive table: location uri contains $specialChars") {
+ // On Windows, it looks colon in the file name is illegal by default. See
+ // https://support.microsoft.com/en-us/help/289627
+ assume(!Utils.isWindows || specialChars != "a:b")
+
withTable("t") {
withTempDir { dir =>
val loc = new File(dir, specialChars)
loc.mkdir()
+ // The parser does not recognize the backslashes on Windows as they
are.
+ // These currently should be escaped.
+ val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\")
spark.sql(
s"""
|CREATE TABLE t(a string)
|USING hive
- |LOCATION '$loc'
+ |LOCATION '$escapedLoc'
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
@@ -1779,12 +1787,13 @@ class HiveDDLSuite
withTempDir { dir =>
val loc = new File(dir, specialChars)
loc.mkdir()
+ val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\")
spark.sql(
s"""
|CREATE TABLE t1(a string, b string)
|USING hive
|PARTITIONED BY(b)
- |LOCATION '$loc'
+ |LOCATION '$escapedLoc'
""".stripMargin)
val table =
spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
@@ -1795,16 +1804,20 @@ class HiveDDLSuite
if (specialChars != "a:b") {
spark.sql("INSERT INTO TABLE t1 PARTITION(b=2) SELECT 1")
val partFile = new File(loc, "b=2")
- assert(partFile.listFiles().length >= 1)
+ assert(partFile.listFiles().nonEmpty)
checkAnswer(spark.table("t1"), Row("1", "2") :: Nil)
spark.sql("INSERT INTO TABLE t1 PARTITION(b='2017-03-03
12:13%3A14') SELECT 1")
val partFile1 = new File(loc, "b=2017-03-03 12:13%3A14")
assert(!partFile1.exists())
- val partFile2 = new File(loc, "b=2017-03-03 12%3A13%253A14")
- assert(partFile2.listFiles().length >= 1)
- checkAnswer(spark.table("t1"),
- Row("1", "2") :: Row("1", "2017-03-03 12:13%3A14") :: Nil)
+
+ if (!Utils.isWindows) {
+ // Actual path becomes "b=2017-03-03%2012%3A13%253A14" on
Windows.
+ val partFile2 = new File(loc, "b=2017-03-03 12%3A13%253A14")
+ assert(partFile2.listFiles().nonEmpty)
+ checkAnswer(spark.table("t1"),
+ Row("1", "2") :: Row("1", "2017-03-03 12:13%3A14") :: Nil)
+ }
} else {
val e = intercept[AnalysisException] {
spark.sql("INSERT INTO TABLE t1 PARTITION(b=2) SELECT 1")
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]