Skip to content

Commit 132cbf0

Browse files
committed
[SPARK-35105][SQL] Support multiple paths for ADD FILE/JAR/ARCHIVE commands
### What changes were proposed in this pull request? This PR extends `ADD FILE/JAR/ARCHIVE` commands to be able to take multiple path arguments like Hive. ### Why are the changes needed? To make those commands more useful. ### Does this PR introduce _any_ user-facing change? Yes. In the current implementation, those commands can take a path which contains whitespaces without enclose it by neither `'` nor `"` but after this change, users need to enclose such paths. I've note this incompatibility in the migration guide. ### How was this patch tested? New tests. Closes #32205 from sarutak/add-multiple-files. Authored-by: Kousuke Saruta <[email protected]> Signed-off-by: Kousuke Saruta <[email protected]>
1 parent 529b875 commit 132cbf0

File tree

10 files changed

+135
-30
lines changed

10 files changed

+135
-30
lines changed

docs/sql-migration-guide.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,8 @@ license: |
2424

2525
## Upgrading from Spark SQL 3.1 to 3.2
2626

27+
- Since Spark 3.2, ADD FILE/JAR/ARCHIVE commands require each path to be enclosed by `"` or `'` if the path contains whitespaces.
28+
2729
- Since Spark 3.2, all the supported JDBC dialects use StringType for ROWID. In Spark 3.1 or earlier, Oracle dialect uses StringType and the other dialects use LongType.
2830

2931
- In Spark 3.2, PostgreSQL JDBC dialect uses StringType for MONEY and MONEY[] is not supported due to the JDBC driver for PostgreSQL can't handle those types properly. In Spark 3.1 or earlier, DoubleType and ArrayType of DoubleType are used respectively.

docs/sql-ref-syntax-aux-resource-mgmt-add-archive.md

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ license: |
2626
### Syntax
2727

2828
```sql
29-
ADD ARCHIVE file_name
29+
ADD { ARCHIVE | ARCHIVES } file_name [ ... ]
3030
```
3131

3232
### Parameters
@@ -42,6 +42,7 @@ ADD ARCHIVE /tmp/test.tar.gz;
4242
ADD ARCHIVE "/path/to/some.zip";
4343
ADD ARCHIVE '/some/other.tgz';
4444
ADD ARCHIVE "/path with space/abc.tar";
45+
ADD ARCHIVES "/path with space/def.tgz" '/path with space/ghi.zip';
4546
```
4647

4748
### Related Statements

docs/sql-ref-syntax-aux-resource-mgmt-add-file.md

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ license: |
2626
### Syntax
2727

2828
```sql
29-
ADD FILE resource_name
29+
ADD { FILE | FILES } resource_name [ ... ]
3030
```
3131

3232
### Parameters
@@ -43,6 +43,7 @@ ADD FILE "/path/to/file/abc.txt";
4343
ADD FILE '/another/test.txt';
4444
ADD FILE "/path with space/abc.txt";
4545
ADD FILE "/path/to/some/directory";
46+
ADD FILES "/path with space/cde.txt" '/path with space/fgh.txt';
4647
```
4748

4849
### Related Statements

docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ license: |
2626
### Syntax
2727

2828
```sql
29-
ADD JAR file_name
29+
ADD { JAR | JARS } file_name [ ... ]
3030
```
3131

3232
### Parameters
@@ -52,6 +52,7 @@ ADD JAR /tmp/test.jar;
5252
ADD JAR "/path/to/some.jar";
5353
ADD JAR '/some/other.jar';
5454
ADD JAR "/path with space/abc.jar";
55+
ADD JARS "/path with space/def.jar" '/path with space/ghi.jar';
5556
ADD JAR "ivy://group:module:version";
5657
ADD JAR "ivy://group:module:version?transitive=false"
5758
ADD JAR "ivy://group:module:version?transitive=true"

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -335,7 +335,7 @@ class SparkSqlAstBuilder extends AstBuilder {
335335
}
336336

337337
/**
338-
* Create a [[AddFileCommand]], [[AddJarCommand]], [[AddArchiveCommand]],
338+
* Create a [[AddFilesCommand]], [[AddJarsCommand]], [[AddArchivesCommand]],
339339
* [[ListFilesCommand]], [[ListJarsCommand]] or [[ListArchivesCommand]]
340340
* command depending on the requested operation on resources.
341341
* Expected format:
@@ -356,15 +356,12 @@ class SparkSqlAstBuilder extends AstBuilder {
356356
case p => p
357357
}
358358

359-
// The implementation of pathForAdd is to keep the compatibility with before SPARK-34977.
360-
val pathForAdd = strLiteralDef.findFirstIn(rawArg)
361-
.find(p => p.startsWith("\"") || p.startsWith("'")).map(unescapeSQLString).getOrElse(rawArg)
362359
ctx.op.getType match {
363360
case SqlBaseParser.ADD =>
364361
ctx.identifier.getText.toLowerCase(Locale.ROOT) match {
365-
case "file" => AddFileCommand(pathForAdd)
366-
case "jar" => AddJarCommand(pathForAdd)
367-
case "archive" => AddArchiveCommand(pathForAdd)
362+
case "files" | "file" => AddFilesCommand(maybePaths)
363+
case "jars" | "jar" => AddJarsCommand(maybePaths)
364+
case "archives" | "archive" => AddArchivesCommand(maybePaths)
368365
case other => operationNotAllowed(s"ADD with resource type '$other'", ctx)
369366
}
370367
case SqlBaseParser.LIST =>

sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -27,30 +27,30 @@ import org.apache.spark.util.Utils
2727
/**
2828
* Adds a jar to the current session so it can be used (for UDFs or serdes).
2929
*/
30-
case class AddJarCommand(path: String) extends LeafRunnableCommand {
30+
case class AddJarsCommand(paths: Seq[String]) extends LeafRunnableCommand {
3131
override def run(sparkSession: SparkSession): Seq[Row] = {
32-
sparkSession.sessionState.resourceLoader.addJar(path)
32+
paths.foreach(sparkSession.sessionState.resourceLoader.addJar(_))
3333
Seq.empty[Row]
3434
}
3535
}
3636

3737
/**
3838
* Adds a file to the current session so it can be used.
3939
*/
40-
case class AddFileCommand(path: String) extends LeafRunnableCommand {
40+
case class AddFilesCommand(paths: Seq[String]) extends LeafRunnableCommand {
4141
override def run(sparkSession: SparkSession): Seq[Row] = {
4242
val recursive = !sparkSession.sessionState.conf.addSingleFileInAddFile
43-
sparkSession.sparkContext.addFile(path, recursive)
43+
paths.foreach(sparkSession.sparkContext.addFile(_, recursive))
4444
Seq.empty[Row]
4545
}
4646
}
4747

4848
/**
4949
* Adds an archive to the current session so it can be used.
5050
*/
51-
case class AddArchiveCommand(path: String) extends LeafRunnableCommand {
51+
case class AddArchivesCommand(paths: Seq[String]) extends LeafRunnableCommand {
5252
override def run(sparkSession: SparkSession): Seq[Row] = {
53-
sparkSession.sparkContext.addArchive(path)
53+
paths.foreach(sparkSession.sparkContext.addArchive(_))
5454
Seq.empty[Row]
5555
}
5656
}

sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala

Lines changed: 11 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -245,21 +245,24 @@ class SparkSqlParserSuite extends AnalysisTest {
245245
}
246246

247247
test("manage resources") {
248-
assertEqual("ADD FILE abc.txt", AddFileCommand("abc.txt"))
249-
assertEqual("ADD FILE 'abc.txt'", AddFileCommand("abc.txt"))
250-
assertEqual("ADD FILE \"/path/to/abc.txt\"", AddFileCommand("/path/to/abc.txt"))
248+
assertEqual("ADD FILE abc.txt", AddFilesCommand(Seq("abc.txt")))
249+
assertEqual("ADD FILE 'abc.txt'", AddFilesCommand(Seq("abc.txt")))
250+
assertEqual("ADD FILE \"/path/to/abc.txt\"", AddFilesCommand("/path/to/abc.txt"::Nil))
251251
assertEqual("LIST FILE abc.txt", ListFilesCommand(Array("abc.txt")))
252252
assertEqual("LIST FILE '/path//abc.txt'", ListFilesCommand(Array("/path//abc.txt")))
253253
assertEqual("LIST FILE \"/path2/abc.txt\"", ListFilesCommand(Array("/path2/abc.txt")))
254-
assertEqual("ADD JAR /path2/_2/abc.jar", AddJarCommand("/path2/_2/abc.jar"))
255-
assertEqual("ADD JAR '/test/path_2/jar/abc.jar'", AddJarCommand("/test/path_2/jar/abc.jar"))
256-
assertEqual("ADD JAR \"abc.jar\"", AddJarCommand("abc.jar"))
254+
assertEqual("ADD JAR /path2/_2/abc.jar", AddJarsCommand(Seq("/path2/_2/abc.jar")))
255+
assertEqual("ADD JAR '/test/path_2/jar/abc.jar'",
256+
AddJarsCommand(Seq("/test/path_2/jar/abc.jar")))
257+
assertEqual("ADD JAR \"abc.jar\"", AddJarsCommand(Seq("abc.jar")))
257258
assertEqual("LIST JAR /path-with-dash/abc.jar",
258259
ListJarsCommand(Array("/path-with-dash/abc.jar")))
259260
assertEqual("LIST JAR 'abc.jar'", ListJarsCommand(Array("abc.jar")))
260261
assertEqual("LIST JAR \"abc.jar\"", ListJarsCommand(Array("abc.jar")))
261-
assertEqual("ADD FILE /path with space/abc.txt", AddFileCommand("/path with space/abc.txt"))
262-
assertEqual("ADD JAR /path with space/abc.jar", AddJarCommand("/path with space/abc.jar"))
262+
assertEqual("ADD FILE '/path with space/abc.txt'",
263+
AddFilesCommand(Seq("/path with space/abc.txt")))
264+
assertEqual("ADD JAR '/path with space/abc.jar'",
265+
AddJarsCommand(Seq("/path with space/abc.jar")))
263266
}
264267

265268
test("SPARK-32608: script transform with row format delimit") {

sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -169,10 +169,10 @@ private[hive] object SparkSQLCLIDriver extends Logging {
169169
}
170170

171171
// The class loader of CliSessionState's conf is current main thread's class loader
172-
// used to load jars passed by --jars. One class loader used by AddJarCommand is
172+
// used to load jars passed by --jars. One class loader used by AddJarsCommand is
173173
// sharedState.jarClassLoader which contain jar path passed by --jars in main thread.
174174
// We set CliSessionState's conf class loader to sharedState.jarClassLoader.
175-
// Thus we can load all jars passed by --jars and AddJarCommand.
175+
// Thus we can load all jars passed by --jars and AddJarsCommand.
176176
sessionState.getConf.setClassLoader(SparkSQLEnv.sqlContext.sharedState.jarClassLoader)
177177

178178
// TODO work around for set the log output to console, because the HiveContext

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -186,8 +186,8 @@ private[hive] class HiveClientImpl(
186186
Hive.set(clientLoader.cachedHive.asInstanceOf[Hive])
187187
}
188188
// Hive 2.3 will set UDFClassLoader to hiveConf when initializing SessionState
189-
// since HIVE-11878, and ADDJarCommand will add jars to clientLoader.classLoader.
190-
// For this reason we cannot load the jars added by ADDJarCommand because of class loader
189+
// since HIVE-11878, and ADDJarsCommand will add jars to clientLoader.classLoader.
190+
// For this reason we cannot load the jars added by ADDJarsCommand because of class loader
191191
// got changed. We reset it to clientLoader.ClassLoader here.
192192
state.getConf.setClassLoader(clientLoader.classLoader)
193193
SessionState.start(state)

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala

Lines changed: 102 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -836,7 +836,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
836836
Files.write(file.toPath, "test_file1".getBytes)
837837
val jarFile = new File(dir, "test file.jar")
838838
TestUtils.createJar(Seq(file), jarFile)
839-
sql(s"ADD JAR ${jarFile.getAbsolutePath}")
839+
sql(s"ADD JAR '${jarFile.getAbsolutePath}'")
840840
assert(sql("LIST JARS").
841841
filter(_.getString(0).contains(s"${jarFile.getName}".replace(" ", "%20"))).count() > 0)
842842
}
@@ -964,6 +964,104 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
964964
}
965965
}
966966

967+
test("SPARK-35105: ADD FILES command with multiple files") {
968+
withTempDir { dir =>
969+
val file1 = File.createTempFile("someprefix1", "somesuffix1", dir)
970+
val file2 = File.createTempFile("someprefix2", "somesuffix 2", dir)
971+
val file3 = File.createTempFile("someprefix3", "somesuffix 3", dir)
972+
val file4 = File.createTempFile("someprefix4", "somesuffix4", dir)
973+
974+
Files.write(file1.toPath, "file1".getBytes)
975+
Files.write(file2.toPath, "file2".getBytes)
976+
Files.write(file3.toPath, "file3".getBytes)
977+
Files.write(file4.toPath, "file3".getBytes)
978+
979+
sql(s"ADD FILE ${file1.getAbsolutePath} '${file2.getAbsoluteFile}'")
980+
sql(s"""ADD FILES "${file3.getAbsolutePath}" ${file4.getAbsoluteFile}""")
981+
val listFiles = sql(s"LIST FILES ${file1.getAbsolutePath} " +
982+
s"'${file2.getAbsolutePath}' '${file3.getAbsolutePath}' ${file4.getAbsolutePath}")
983+
assert(listFiles.count === 4)
984+
assert(listFiles.filter(_.getString(0).contains(file1.getName)).count() === 1)
985+
assert(listFiles.filter(
986+
_.getString(0).contains(file2.getName.replace(" ", "%20"))).count() === 1)
987+
assert(listFiles.filter(
988+
_.getString(0).contains(file3.getName.replace(" ", "%20"))).count() === 1)
989+
assert(listFiles.filter(_.getString(0).contains(file4.getName)).count() === 1)
990+
}
991+
}
992+
993+
test("SPARK-35105: ADD JARS command with multiple files") {
994+
withTempDir { dir =>
995+
val file1 = new File(dir, "test1.txt")
996+
val file2 = new File(dir, "test2.txt")
997+
val file3 = new File(dir, "test3.txt")
998+
val file4 = new File(dir, "test4.txt")
999+
1000+
Files.write(file1.toPath, "file1".getBytes)
1001+
Files.write(file2.toPath, "file2".getBytes)
1002+
Files.write(file3.toPath, "file3".getBytes)
1003+
Files.write(file4.toPath, "file4".getBytes)
1004+
1005+
val jarFile1 = File.createTempFile("someprefix1", "somesuffix 1", dir)
1006+
val jarFile2 = File.createTempFile("someprefix2", "somesuffix2", dir)
1007+
val jarFile3 = File.createTempFile("someprefix3", "somesuffix3", dir)
1008+
val jarFile4 = File.createTempFile("someprefix4", "somesuffix 4", dir)
1009+
1010+
TestUtils.createJar(Seq(file1), jarFile1)
1011+
TestUtils.createJar(Seq(file2), jarFile2)
1012+
TestUtils.createJar(Seq(file3), jarFile3)
1013+
TestUtils.createJar(Seq(file4), jarFile4)
1014+
1015+
sql(s"""ADD JAR "${jarFile1.getAbsolutePath}" ${jarFile2.getAbsoluteFile}""")
1016+
sql(s"ADD JARS ${jarFile3.getAbsolutePath} '${jarFile4.getAbsoluteFile}'")
1017+
val listFiles = sql(s"LIST JARS '${jarFile1.getAbsolutePath}' " +
1018+
s"${jarFile2.getAbsolutePath} ${jarFile3.getAbsolutePath} '${jarFile4.getAbsoluteFile}'")
1019+
assert(listFiles.count === 4)
1020+
assert(listFiles.filter(
1021+
_.getString(0).contains(jarFile1.getName.replace(" ", "%20"))).count() === 1)
1022+
assert(listFiles.filter(_.getString(0).contains(jarFile2.getName)).count() === 1)
1023+
assert(listFiles.filter(_.getString(0).contains(jarFile3.getName)).count() === 1)
1024+
assert(listFiles.filter(
1025+
_.getString(0).contains(jarFile4.getName.replace(" ", "%20"))).count() === 1)
1026+
}
1027+
}
1028+
1029+
test("SPARK-35105: ADD ARCHIVES command with multiple files") {
1030+
withTempDir { dir =>
1031+
val file1 = new File(dir, "test1.txt")
1032+
val file2 = new File(dir, "test2.txt")
1033+
val file3 = new File(dir, "test3.txt")
1034+
val file4 = new File(dir, "test4.txt")
1035+
1036+
Files.write(file1.toPath, "file1".getBytes)
1037+
Files.write(file2.toPath, "file2".getBytes)
1038+
Files.write(file3.toPath, "file3".getBytes)
1039+
Files.write(file4.toPath, "file4".getBytes)
1040+
1041+
val jarFile1 = File.createTempFile("someprefix1", "somesuffix1", dir)
1042+
val jarFile2 = File.createTempFile("someprefix2", "somesuffix 2", dir)
1043+
val jarFile3 = File.createTempFile("someprefix3", "somesuffix3", dir)
1044+
val jarFile4 = File.createTempFile("someprefix4", "somesuffix 4", dir)
1045+
1046+
TestUtils.createJar(Seq(file1), jarFile1)
1047+
TestUtils.createJar(Seq(file2), jarFile2)
1048+
TestUtils.createJar(Seq(file3), jarFile3)
1049+
TestUtils.createJar(Seq(file4), jarFile4)
1050+
1051+
sql(s"""ADD ARCHIVE ${jarFile1.getAbsolutePath} "${jarFile2.getAbsoluteFile}"""")
1052+
sql(s"ADD ARCHIVES ${jarFile3.getAbsolutePath} '${jarFile4.getAbsoluteFile}'")
1053+
val listFiles = sql(s"LIST ARCHIVES ${jarFile1.getAbsolutePath} " +
1054+
s"'${jarFile2.getAbsolutePath}' ${jarFile3.getAbsolutePath} '${jarFile4.getAbsolutePath}'")
1055+
assert(listFiles.count === 4)
1056+
assert(listFiles.filter(_.getString(0).contains(jarFile1.getName)).count() === 1)
1057+
assert(listFiles.filter(
1058+
_.getString(0).contains(jarFile2.getName.replace(" ", "%20"))).count() === 1)
1059+
assert(listFiles.filter(_.getString(0).contains(jarFile3.getName)).count() === 1)
1060+
assert(listFiles.filter(
1061+
_.getString(0).contains(jarFile4.getName.replace(" ", "%20"))).count() === 1)
1062+
}
1063+
}
1064+
9671065
test("SPARK-34977: LIST FILES/JARS/ARCHIVES should handle multiple quoted path arguments") {
9681066
withTempDir { dir =>
9691067
val file1 = File.createTempFile("someprefix1", "somesuffix1", dir)
@@ -979,6 +1077,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
9791077
sql(s"ADD FILE '${file3.getAbsolutePath}'")
9801078
val listFiles = sql("LIST FILES " +
9811079
s"""'${file1.getAbsolutePath}' ${file2.getAbsolutePath} "${file3.getAbsolutePath}"""")
1080+
9821081
assert(listFiles.count === 3)
9831082
assert(listFiles.filter(_.getString(0).contains(file1.getName)).count() === 1)
9841083
assert(listFiles.filter(_.getString(0).contains(file2.getName)).count() === 1)
@@ -1004,6 +1103,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
10041103
sql(s"ADD ARCHIVE '${jarFile3.getAbsolutePath}'")
10051104
val listArchives = sql(s"LIST ARCHIVES '${jarFile1.getAbsolutePath}' " +
10061105
s"""${jarFile2.getAbsolutePath} "${jarFile3.getAbsolutePath}"""")
1106+
10071107
assert(listArchives.count === 3)
10081108
assert(listArchives.filter(_.getString(0).contains(jarFile1.getName)).count() === 1)
10091109
assert(listArchives.filter(_.getString(0).contains(jarFile2.getName)).count() === 1)
@@ -1026,7 +1126,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
10261126

10271127
sql(s"ADD JAR ${jarFile4.getAbsolutePath}")
10281128
sql(s"ADD JAR ${jarFile5.getAbsolutePath}")
1029-
sql(s"ADD JAR ${jarFile6.getAbsolutePath}")
1129+
sql(s"ADD JAR '${jarFile6.getAbsolutePath}'")
10301130
val listJars = sql(s"LIST JARS '${jarFile4.getAbsolutePath}' " +
10311131
s"""${jarFile5.getAbsolutePath} "${jarFile6.getAbsolutePath}"""")
10321132
assert(listJars.count === 3)

0 commit comments

Comments
 (0)