This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.1 by this push:
     new 6ca8e75  [SPARK-33438][SQL] Eagerly init objects with defined SQL 
Confs for command `set -v`
6ca8e75 is described below

commit 6ca8e75592da52eba8b033a02aafa5045a69f8ed
Author: Linhong Liu <linhong....@databricks.com>
AuthorDate: Mon Feb 8 22:48:28 2021 +0900

    [SPARK-33438][SQL] Eagerly init objects with defined SQL Confs for command 
`set -v`
    
    ### What changes were proposed in this pull request?
    In Spark, `set -v` is defined as "Queries all properties that are defined 
in the SQLConf of the sparkSession".
    But there are other external modules that also define properties and 
register them to SQLConf. In this case,
    it can't be displayed by `set -v` until the conf object is initiated (i.e. 
calling the object at least once).
    
    In this PR, I propose to eagerly initiate all the objects registered to 
SQLConf, so that `set -v` will always output
    the completed properties.
    
    ### Why are the changes needed?
    Improve the `set -v` command to produces completed and  deterministic 
results
    
    ### Does this PR introduce _any_ user-facing change?
    `set -v` command will dump more configs
    
    ### How was this patch tested?
    existing tests
    
    Closes #30363 from linhongliu-db/set-v.
    
    Authored-by: Linhong Liu <linhong....@databricks.com>
    Signed-off-by: HyukjinKwon <gurwls...@apache.org>
    (cherry picked from commit 037bfb2dbcb73cfbd73f0fd9abe0b38789a182a2)
    Signed-off-by: HyukjinKwon <gurwls...@apache.org>
---
 .../org/apache/spark/sql/internal/SQLConf.scala    | 24 ++++++++++++++++++++++
 .../spark/sql/api/python/PythonSQLUtils.scala      | 15 +-------------
 2 files changed, 25 insertions(+), 14 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 55c7016..ecd8f3a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -25,6 +25,7 @@ import java.util.zip.Deflater
 import scala.collection.JavaConverters._
 import scala.collection.immutable
 import scala.util.Try
+import scala.util.control.NonFatal
 import scala.util.matching.Regex
 
 import org.apache.hadoop.fs.Path
@@ -35,6 +36,7 @@ import org.apache.spark.internal.config._
 import org.apache.spark.internal.config.{IGNORE_MISSING_FILES => 
SPARK_IGNORE_MISSING_FILES}
 import org.apache.spark.network.util.ByteUnit
 import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver}
 import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode
 import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator
@@ -3768,6 +3770,27 @@ class SQLConf extends Serializable with Logging {
     }
   }
 
+  private var definedConfsLoaded = false
+  /**
+   * Init [[StaticSQLConf]] and [[org.apache.spark.sql.hive.HiveUtils]] so 
that all the defined
+   * SQL Configurations will be registered to SQLConf
+   */
+  private def loadDefinedConfs(): Unit = {
+    if (!definedConfsLoaded) {
+      definedConfsLoaded = true
+      // Force to register static SQL configurations
+      StaticSQLConf
+      try {
+        // Force to register SQL configurations from Hive module
+        val symbol = 
ScalaReflection.mirror.staticModule("org.apache.spark.sql.hive.HiveUtils")
+        ScalaReflection.mirror.reflectModule(symbol).instance
+      } catch {
+        case NonFatal(e) =>
+          logWarning("SQL configurations from Hive module is not loaded", e)
+      }
+    }
+  }
+
   /**
    * Return all the configuration properties that have been set (i.e. not the 
default).
    * This creates a new copy of the config properties in the form of a Map.
@@ -3780,6 +3803,7 @@ class SQLConf extends Serializable with Logging {
    * definition contains key, defaultValue and doc.
    */
   def getAllDefinedConfs: Seq[(String, String, String, String)] = 
sqlConfEntries.synchronized {
+    loadDefinedConfs()
     sqlConfEntries.values.asScala.filter(_.isPublic).map { entry =>
       val displayValue = Option(getConfString(entry.key, 
null)).getOrElse(entry.defaultValueString)
       (entry.key, displayValue, entry.doc, entry.version)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
index 3825460..2cd26e2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
@@ -20,20 +20,17 @@ package org.apache.spark.sql.api.python
 import java.io.InputStream
 import java.nio.channels.Channels
 
-import scala.util.control.NonFatal
-
 import org.apache.spark.api.java.JavaRDD
 import org.apache.spark.api.python.PythonRDDServer
 import org.apache.spark.internal.Logging
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{DataFrame, SQLContext}
-import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
 import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
 import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
 import org.apache.spark.sql.execution.{ExplainMode, QueryExecution}
 import org.apache.spark.sql.execution.arrow.ArrowConverters
-import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.DataType
 
 private[sql] object PythonSQLUtils extends Logging {
@@ -46,16 +43,6 @@ private[sql] object PythonSQLUtils extends Logging {
 
   private def listAllSQLConfigs(): Seq[(String, String, String, String)] = {
     val conf = new SQLConf()
-    // Force to build static SQL configurations
-    StaticSQLConf
-    // Force to build SQL configurations from Hive module
-    try {
-      val symbol = 
ScalaReflection.mirror.staticModule("org.apache.spark.sql.hive.HiveUtils")
-      ScalaReflection.mirror.reflectModule(symbol).instance
-    } catch {
-      case NonFatal(e) =>
-        logWarning("Cannot generated sql configurations from hive module", e)
-    }
     conf.getAllDefinedConfs
   }
 


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

Reply via email to