AngersZhuuuu commented on a change in pull request #29881:
URL: https://github.com/apache/spark/pull/29881#discussion_r504668188
##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
##########
@@ -396,6 +419,93 @@ private[spark] object HiveUtils extends Logging {
config = configurations,
barrierPrefixes = hiveMetastoreBarrierPrefixes,
sharedPrefixes = hiveMetastoreSharedPrefixes)
+ } else if (hiveMetastoreJars == "path") {
+
+ def addLocalHiveJars(file: File): Seq[URL] = {
+ if (file.getName == "*") {
+ val files = file.getParentFile.listFiles()
+ if (files == null) {
+ logWarning(s"Hive jar path '${file.getPath}' does not exist.")
+ Nil
+ } else {
+
files.filter(_.getName.toLowerCase(Locale.ROOT).endsWith(".jar")).map(_.toURL).toSeq
+ }
+ } else {
+ file.toURL :: Nil
+ }
+ }
+
+ def checkRemoteHiveJars(path: String): Seq[URL] = {
+ try {
+ val hadoopPath = new Path(path)
+ val fs = hadoopPath.getFileSystem(hadoopConf)
+ if (hadoopPath.getName == "*") {
+ val parent = hadoopPath.getParent
+ if (!fs.exists(parent)) {
+ logWarning(s"Hive Jar ${path} does not exist.")
+ Nil
+ } else if (!fs.getFileStatus(parent).isDirectory) {
+ logWarning(s"Hive Jar ${parent} is not a directory.")
+ Nil
+ } else {
+ fs.listStatus(parent).map(_.getPath.toUri.toURL)
+ }
+ } else {
+ if (!fs.exists(hadoopPath)) {
+ logWarning(s"Hive Jar ${path} does not exist.")
+ Nil
+ } else if (fs.getFileStatus(hadoopPath).isDirectory) {
+ logWarning(s"Hive Jar ${path} not allow directory without `*`")
+ Nil
+ } else {
+ // Since tar/tar.gz file we can't know it's final path yet, not
support it
+ hadoopPath.toUri.toURL :: Nil
+ }
+ }
+ } catch {
+ case NonFatal(e) =>
+ logError(s"Failed to find $path to Hive Jars", e)
+ Nil
+ }
+ }
+
+ // Convert to files and expand any directories.
+ val jars =
+ HiveUtils.hiveMetastoreJarsPath(sqlConf)
+ .flatMap {
+ case path if Utils.isWindows =>
+ addLocalHiveJars(new File(path))
+ case path =>
+ val uri = new Path(path).toUri
+ uri.getScheme match {
+ case "file" | "local" =>
Review comment:
> IIUC the Hadoop DFS API also supports local file systems.
Not support local file system, but it can. support `http/https/ftp`
```
~/spark-3.1.0$ bin/spark-sql --master local --conf
spark.sql.hive.metastore.jars=path --conf
spark.sql.hive.metastore.jars.path=/home/hadoop/.ivy2/jars/* --conf
spark.sql.hive.metastore.version=2.3.7 --conf
spark.driver.extraClassPath=$HIVE_CONF_DIR
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use
setLogLevel(newLevel).
Exception in thread "main" org.apache.spark.sql.AnalysisException: Path does
not exist: hdfs://redpoll/home/hadoop/.ivy2/jars/*;
at
org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$3(DataSource.scala:790)
at
org.apache.spark.util.ThreadUtils$.$anonfun$parmap$2(ThreadUtils.scala:373)
at scala.concurrent.Future$.$anonfun$apply$1(Future.scala:659)
at scala.util.Success.$anonfun$map$1(Try.scala:255)
at scala.util.Success.map(Try.scala:213)
at scala.concurrent.Future.$anonfun$map$1(Future.scala:292)
at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:33)
at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:33)
at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64)
at
java.util.concurrent.ForkJoinTask$RunnableExecuteAction.exec(ForkJoinTask.java:1402)
at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
at
java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
at
java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157)
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]