You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/10/12 03:51:14 UTC

[GitHub] [spark] sunchao commented on a change in pull request #29881: [SPARK-32852][SQL] spark.sql.hive.metastore.jars support HDFS location

sunchao commented on a change in pull request #29881:
URL: https://github.com/apache/spark/pull/29881#discussion_r502744398



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
##########
@@ -88,12 +90,23 @@ private[spark] object HiveUtils extends Logging {
       |   <code>${builtinHiveVersion}</code> or not defined.
       | 2. "maven"
       |   Use Hive jars of specified version downloaded from Maven repositories.
-      | 3. A classpath in the standard format for both Hive and Hadoop.
+      | 3. "path"
+      |   Use Hive jars configured by `spark.sql.hive.metastore.jars.path` in comma separated format
+      |   support both local or remote paths.
+      | 4. A classpath in the standard format for both Hive and Hadoop, we should always
+      |   be fully qualified URL to indicate other file systems.
       """.stripMargin)
     .version("1.4.0")
     .stringConf
     .createWithDefault("builtin")
 
+  val HIVE_METASTORE_JARS_PATH = buildStaticConf("spark.sql.hive.metastore.jars.path")
+    .doc(s"Comma separated path of Hive jars, both support local and remote paths." +

Review comment:
       nit: `both support` -> `support both`.

##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
##########
@@ -88,12 +90,23 @@ private[spark] object HiveUtils extends Logging {
       |   <code>${builtinHiveVersion}</code> or not defined.
       | 2. "maven"
       |   Use Hive jars of specified version downloaded from Maven repositories.
-      | 3. A classpath in the standard format for both Hive and Hadoop.
+      | 3. "path"
+      |   Use Hive jars configured by `spark.sql.hive.metastore.jars.path` in comma separated format
+      |   support both local or remote paths.
+      | 4. A classpath in the standard format for both Hive and Hadoop, we should always

Review comment:
       nit: `we should always` -> `it should always`?

##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
##########
@@ -396,6 +417,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

Review comment:
       How are these jars be downloaded eventually? we'll need to call `FileSystem` API for that later right?

##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
##########
@@ -396,6 +417,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)

Review comment:
       Do we handle nested `*`, e.g., `path = `/foo/*/*`?

##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
##########
@@ -397,27 +399,86 @@ private[spark] object HiveUtils extends Logging {
         barrierPrefixes = hiveMetastoreBarrierPrefixes,
         sharedPrefixes = hiveMetastoreSharedPrefixes)
     } else {
-      // Convert to files and expand any directories.
-      val jars =
-        hiveMetastoreJars
-          .split(File.pathSeparator)
-          .flatMap {
-          case path if new File(path).getName == "*" =>
-            val files = new File(path).getParentFile.listFiles()
-            if (files == null) {
-              logWarning(s"Hive jar path '$path' does not exist.")
+
+      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 {
-              files.filter(_.getName.toLowerCase(Locale.ROOT).endsWith(".jar")).toSeq
+              fs.listStatus(parent).map(_.getPath.toUri.toURL)
             }
-          case path =>
-            new File(path) :: Nil
+          } 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
         }
-          .map(_.toURI.toURL)
+      }
+
+      // Convert to files and expand any directories.
+      val jars =
+        hiveMetastoreJars
+          .split(";")
+          .flatMap {
+            case path if path.contains("\\") =>
+              addLocalHiveJars(new File(path))
+            case path =>
+              val uri = new Path(path).toUri
+              uri.getScheme match {
+                case null | "file" =>
+                  addLocalHiveJars(new File(uri.getPath))

Review comment:
       Should this comment on the option "path" instead of "classpath"?




----------------------------------------------------------------
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:
users@infra.apache.org



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