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 2022/08/26 19:41:48 UTC

[GitHub] [spark] roczei opened a new pull request, #37679: [SPARK-35242][FOLLOWUP][Ranger][Hive][default db] Spark should not rely on the 'default' hive database.

roczei opened a new pull request, #37679:
URL: https://github.com/apache/spark/pull/37679

   ### What changes were proposed in this pull request?
   
   This PR is a follow-up PR for SPARK-37731. Previous has been closed by github-actions: https://github.com/apache/spark/pull/32364
   
   My changes:
   
   - Rebased / updated the previous PR to the latest master branch version
   - Deleted the DEFAULT_DATABASE  static member from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala and refactored the code regarding this
   
   ### Why are the changes needed?
   
   If our user does not have any permissions for the Hive default database in Ranger, it will fail with the following error:
   
   ```
   22/08/26 18:36:21 INFO  metastore.RetryingMetaStoreClient: [main]: RetryingMetaStoreClient proxy=class org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient ugi=hrt_10@ROOT.HWX.SITE (auth:KERBEROS) retries=1 delay=1 lifetime=0
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:Permission denied: user [hrt_10] does not have [USE] privilege on [default])
     at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:110)
     at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:223)
     at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:150)
     at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:144)
   ```
   The idea is that we introduce a new configuration parameter where we can set a different database name for the default database. Our user has enough permissions for this  in Ranger.
   
   For example:
   
   ```spark-shell --conf spark.sql.catalog.spark_catalog.defaultDatabase=other_db```
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   
   There will be a new configuration parameter as I mentioned above but the default value is "default" as it was previously.
   
   
   ### How was this patch tested?
   
   1) With github action (all tests passed)
   
   https://github.com/roczei/spark/actions/runs/2935626152
   
   2) Manually tested with Ranger + Hive
   
   Scenario a) hrt_10 does not have access to the default database in Hive: 
   
   
   ```
   [hrt_10@quasar-thbnqr-2 ~]$ spark-shell
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/08/26 18:14:18 WARN  conf.HiveConf: [main]: HiveConf of name hive.masking.algo does not exist
   22/08/26 18:14:30 WARN  cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: [dispatcher-event-loop-17]: Attempted to request executors before the AM has registered!
   
   
   ...
   
   scala> spark.sql("use other")
   22/08/26 18:18:47 INFO  conf.HiveConf: [main]: Found configuration file file:/etc/hive/conf/hive-site.xml
   22/08/26 18:18:48 WARN  conf.HiveConf: [main]: HiveConf of name hive.masking.algo does not exist
   22/08/26 18:18:48 WARN  client.HiveClientImpl: [main]: Detected HiveConf hive.execution.engine is 'tez' and will be reset to 'mr' to disable useless hive logic
   Hive Session ID = 2188764e-d0dc-41b3-b714-f89b03cb3d6d
   22/08/26 18:18:48 INFO  SessionState: [main]: Hive Session ID = 2188764e-d0dc-41b3-b714-f89b03cb3d6d
   22/08/26 18:18:50 INFO  metastore.HiveMetaStoreClient: [main]: HMS client filtering is enabled.
   22/08/26 18:18:50 INFO  metastore.HiveMetaStoreClient: [main]: Trying to connect to metastore with URI thrift://quasar-thbnqr-4.quasar-thbnqr.root.hwx.site:9083
   22/08/26 18:18:50 INFO  metastore.HiveMetaStoreClient: [main]: HMSC::open(): Could not find delegation token. Creating KERBEROS-based thrift connection.
   22/08/26 18:18:50 INFO  metastore.HiveMetaStoreClient: [main]: Opened a connection to metastore, current connections: 1
   22/08/26 18:18:50 INFO  metastore.HiveMetaStoreClient: [main]: Connected to metastore.
   22/08/26 18:18:50 INFO  metastore.RetryingMetaStoreClient: [main]: RetryingMetaStoreClient proxy=class org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient ugi=hrt_10@ROOT.HWX.SITE (auth:KERBEROS) retries=1 delay=1 lifetime=0
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:Permission denied: user [hrt_10] does not have [USE] privilege on [default])
     at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:110)
     at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:223)
     at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:150)
     at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:144)
     at org.apache.spark.sql.internal.SharedState.globalTempViewManager$lzycompute(SharedState.scala:179)
   ```
   
   This is the expected behavior because it will use the "default" db name.
   
   Scenario b) Use the "other" database where the hrt_10 user has proper permissions  
   
   ```
   [hrt_10@quasar-thbnqr-2 ~]$ spark3-shell --conf spark.sql.catalog.spark_catalog.defaultDatabase=other
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/08/26 18:27:03 WARN  conf.HiveConf: [main]: HiveConf of name hive.masking.algo does not exist
   22/08/26 18:27:14 WARN  cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: [dispatcher-event-loop-15]: Attempted to request executors before the AM has registered!
   
   ...
   
   scala> spark.sql("use other")
   22/08/26 18:29:22 INFO  conf.HiveConf: [main]: Found configuration file file:/etc/hive/conf/hive-site.xml
   22/08/26 18:29:22 WARN  conf.HiveConf: [main]: HiveConf of name hive.masking.algo does not exist
   22/08/26 18:29:22 WARN  client.HiveClientImpl: [main]: Detected HiveConf hive.execution.engine is 'tez' and will be reset to 'mr' to disable useless hive logic
   Hive Session ID = 47721693-dbfe-4760-80f6-d4a76a3b37d2
   22/08/26 18:29:22 INFO  SessionState: [main]: Hive Session ID = 47721693-dbfe-4760-80f6-d4a76a3b37d2
   22/08/26 18:29:24 INFO  metastore.HiveMetaStoreClient: [main]: HMS client filtering is enabled.
   22/08/26 18:29:24 INFO  metastore.HiveMetaStoreClient: [main]: Trying to connect to metastore with URI thrift://quasar-thbnqr-4.quasar-thbnqr.root.hwx.site:9083
   22/08/26 18:29:24 INFO  metastore.HiveMetaStoreClient: [main]: HMSC::open(): Could not find delegation token. Creating KERBEROS-based thrift connection.
   22/08/26 18:29:24 INFO  metastore.HiveMetaStoreClient: [main]: Opened a connection to metastore, current connections: 1
   22/08/26 18:29:24 INFO  metastore.HiveMetaStoreClient: [main]: Connected to metastore.
   22/08/26 18:29:24 INFO  metastore.RetryingMetaStoreClient: [main]: RetryingMetaStoreClient proxy=class org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient ugi=hrt_10@ROOT.HWX.SITE (auth:KERBEROS) retries=1 delay=1 lifetime=0
   res0: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("select * from employee").show()
   +---+----+------+-----------+                                                   
   |eid|name|salary|destination|
   +---+----+------+-----------+
   | 12| Ram|    10|     Szeged|
   | 13| Joe|    20|   Debrecen|
   +---+----+------+-----------+
   
   
   scala>
   ```


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1250982553

   SGTM!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r967420952


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -286,7 +284,7 @@ class SessionCatalog(
 
   def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
     val dbName = format(db)
-    if (dbName == DEFAULT_DATABASE) {
+    if (dbName == defaultDatabase) {

Review Comment:
   Here is an example where the user specified default database is "other". Currently if we try to drop "other", it will fail with "Can not drop default database" message. 
   
   ```
   $ bin/spark-shell --conf spark.sql.catalogImplementation=hive --conf spark.sql.catalog.spark_catalog.defaultDatabase=other
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1662751254354).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 1.8.0_345)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   +---------+
   |namespace|
   +---------+
   |  default|
   |    other|
   +---------+
   
   
   scala> spark.sql("drop database other")
   org.apache.spark.sql.AnalysisException: Can not drop default database
     at org.apache.spark.sql.errors.QueryCompilationErrors$.cannotDropDefaultDatabaseError(QueryCompilationErrors.scala:635)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.dropDatabase(SessionCatalog.scala:288)
     at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.dropNamespace(V2SessionCatalog.scala:299)
     at org.apache.spark.sql.execution.datasources.v2.DropNamespaceExec.run(DropNamespaceExec.scala:42)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:111)
     at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:171)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
     at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
     at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:97)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
   ```
   
   I am not sure that I fully understand the request here. Would you like to allow this "drop database other"? Please elaborate a bit.
   
   There was a previous Spark jira to prevent dropping current database, so probably we should keep this. Related jira:
   
   Prevent dropping current database: 
   
   - https://issues.apache.org/jira/browse/SPARK-16459
   - https://github.com/apache/spark/pull/14115



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1251364841

   Thanks @cloud-fan, I have implemented this and all tests passed. As I see we have resolved all of your feedbacks.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1249421287

   > override val defaultNamespace: Array[String] = Array(SQLConf.get.defaultDatabase)
   
   Yes


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1242406713

   > we should also update `V2SessionCatalog.defaultNamespace`
   
   @cloud-fan, for example this change will be good?
   
   ```
   diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
   index efbc9dd755..23775c3ae0 100644
   --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
   +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
   @@ -33,6 +33,7 @@ import org.apache.spark.sql.connector.expressions.Transform
    import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
    import org.apache.spark.sql.execution.datasources.DataSource
    import org.apache.spark.sql.internal.connector.V1Function
   +import org.apache.spark.sql.internal.SQLConf
    import org.apache.spark.sql.types.StructType
    import org.apache.spark.sql.util.CaseInsensitiveStringMap
    
   @@ -43,7 +44,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
      extends TableCatalog with FunctionCatalog with SupportsNamespaces with SQLConfHelper {
      import V2SessionCatalog._
    
   -  override val defaultNamespace: Array[String] = Array("default")
   +  override val defaultNamespace: Array[String] = Array(SQLConf.get.defaultDatabase)
   ```


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r977124552


##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,18 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if (SessionCatalog.DEFAULT_DATABASE != SQLConf.get.defaultDatabase) {
+        throw new SparkException(s"Default catalog database '${SQLConf.get.defaultDatabase}' " +
+          s"not exist, please create it first or change default database to 'default'.")

Review Comment:
   let's use the new error framework: define the error in `error-classes.json`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r977124092


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala:
##########
@@ -43,7 +44,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
   extends TableCatalog with FunctionCatalog with SupportsNamespaces with SQLConfHelper {
   import V2SessionCatalog._
 
-  override val defaultNamespace: Array[String] = Array("default")
+  override val defaultNamespace: Array[String] = Array(SQLConf.get.defaultDatabase)

Review Comment:
   `catalog.defaultDatabase`? It's already available in `SessionCatalog`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1240616362

   Thanks @cloud-fan, I will check these!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r973767185


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -286,7 +284,7 @@ class SessionCatalog(
 
   def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
     val dbName = format(db)
-    if (dbName == DEFAULT_DATABASE) {
+    if (dbName == defaultDatabase) {

Review Comment:
   @cloud-fan,
   
   If the default database is xyz and the current database is abc, we cannot drop the xyz database. Here are my validation steps:
   
   1)
   
   Test setup where the default database is the "default" and created the abc and the xyz databases:
   
   ```
   $ ./spark-shell --conf spark.sql.catalogImplementation=hive
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/18 21:23:12 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1663528992732).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 11.0.16)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("create database xyz")
   22/09/18 21:23:19 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
   22/09/18 21:23:19 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
   22/09/18 21:23:21 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
   22/09/18 21:23:21 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore roczei@192.168.0.38
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database default, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database xyz, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database xyz, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database global_temp, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database xyz, returning NoSuchObjectException
   res0: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("create database abc")
   22/09/18 21:23:40 WARN ObjectStore: Failed to get database abc, returning NoSuchObjectException
   22/09/18 21:23:40 WARN ObjectStore: Failed to get database abc, returning NoSuchObjectException
   22/09/18 21:23:40 WARN ObjectStore: Failed to get database abc, returning NoSuchObjectException
   res1: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("show databases").show()
   +---------+
   |namespace|
   +---------+
   |      abc|
   |  default|
   |      xyz|
   +---------+
   
   scala> :quit
   ```
   
   
   2)
   
   ```
   $ ./spark-shell --conf spark.sql.catalogImplementation=hive --conf spark.sql.catalog.spark_catalog.defaultDatabase=xyz
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/18 21:24:05 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1663529046120).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 11.0.16)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   22/09/18 21:24:11 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
   22/09/18 21:24:11 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
   22/09/18 21:24:13 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
   22/09/18 21:24:13 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore roczei@192.168.0.38
   +---------+
   |namespace|
   +---------+
   |      abc|
   |  default|
   |      xyz|
   +---------+
   
   scala> spark.sql("use database abc")
   22/09/18 21:42:11 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
   22/09/18 21:42:11 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
   22/09/18 21:42:12 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
   22/09/18 21:42:12 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore roczei@192.168.0.38
   22/09/18 21:42:12 WARN ObjectStore: Failed to get database global_temp, returning NoSuchObjectException
   res0: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("SELECT current_database() AS db").show()
   +---+
   | db|
   +---+
   |abc|
   +---+
   
   
   scala> spark.sql("drop database xyz")
   org.apache.spark.sql.AnalysisException: Can not drop default database
     at org.apache.spark.sql.errors.QueryCompilationErrors$.cannotDropDefaultDatabaseError(QueryCompilationErrors.scala:635)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.dropDatabase(SessionCatalog.scala:288)
     at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.dropNamespace(V2SessionCatalog.scala:300)
     at org.apache.spark.sql.execution.datasources.v2.DropNamespaceExec.run(DropNamespaceExec.scala:42)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:111)
     at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:171)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
     at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
     at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:97)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r975366440


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala:
##########
@@ -33,7 +33,7 @@ class CatalogManagerSuite extends SparkFunSuite with SQLHelper {
   private def createSessionCatalog(): SessionCatalog = {
     val catalog = new V1InMemoryCatalog()
     catalog.createDatabase(
-      CatalogDatabase(SessionCatalog.DEFAULT_DATABASE, "", new URI("fake"), Map.empty),
+      CatalogDatabase(SQLConf.get.defaultDatabase, "", new URI("fake"), Map.empty),

Review Comment:
   It's test, I think we can still use `SessionCatalog.DEFAULT_DATABASE`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r978186527


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala:
##########
@@ -43,7 +44,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
   extends TableCatalog with FunctionCatalog with SupportsNamespaces with SQLConfHelper {
   import V2SessionCatalog._
 
-  override val defaultNamespace: Array[String] = Array("default")
+  override val defaultNamespace: Array[String] = Array(SQLConf.get.defaultDatabase)

Review Comment:
   catalog.defaultDatabase results the following compilation error, therefore I have not modified this part:
   
   ```
   [ERROR] /Users/roczei/github/CDPD-40362/upstream/spark/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala:46: value defaultDatabase is not a member of org.apache.spark.sql.catalyst.catalog.SessionCatalog
   [ERROR] one error found
   ```
   
   



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan closed pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
URL: https://github.com/apache/spark/pull/37679


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1257184837

   Hi @cloud-fan,
   
   All build issues have been fixed and all of your feedbacks have been implemented. Latest state:
   
   ```
   $ bin/spark-shell --conf spark.sql.catalog.spark_catalog.defaultDatabase=other_db
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/25 14:34:16 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1664109256655).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.17 (OpenJDK 64-Bit Server VM, Java 1.8.0_345)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   org.apache.spark.SparkDefaultDatabaseNotExistsException: [DEFAULT_DATABASE_NOT_EXISTS] Default database other_db not exist, please create it first or change default database to 'default'.
     at org.apache.spark.sql.errors.QueryExecutionErrors$.defaultDatabaseNotExistsError(QueryExecutionErrors.scala:1936)
     at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:156)
     at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:147)
     at org.apache.spark.sql.internal.BaseSessionStateBuilder.$anonfun$catalog$1(BaseSessionStateBuilder.scala:154)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog$lzycompute(SessionCatalog.scala:122)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog(SessionCatalog.scala:122)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.listDatabases(SessionCatalog.scala:323)
     at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.listNamespaces(V2SessionCatalog.scala:232)
     at org.apache.spark.sql.execution.datasources.v2.ShowNamespacesExec.run(ShowNamespacesExec.scala:42)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:111)
     at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:171)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
     at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
   ``` 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r974888079


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -48,9 +48,6 @@ import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils}
 import org.apache.spark.util.Utils
 
-object SessionCatalog {
-  val DEFAULT_DATABASE = "default"

Review Comment:
   can we still keep it so that we don't need to hardcode "default"?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1250375409

   > > override val defaultNamespace: Array[String] = Array(SQLConf.get.defaultDatabase)
   > 
   > Yes
   
   @cloud-fan Ok, I have fixed this.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r973875374


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -286,7 +284,7 @@ class SessionCatalog(
 
   def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
     val dbName = format(db)
-    if (dbName == DEFAULT_DATABASE) {
+    if (dbName == defaultDatabase) {

Review Comment:
   Is this a common behavior in other databases?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r979572317


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1932,6 +1932,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
     new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast")
   }
 
+  def defaultDatabaseNotExistsError(defaultDatabase: String): Throwable = {
+    new SparkDefaultDatabaseNotExistsException(

Review Comment:
   We don't need a new exception. We can just use `SparkException`
   ```
   new SparkException(
     errorClass = "DEFAULT_DATABASE_NOT_EXISTS",
     messageParameters = Map("defaultDatabase" -> defaultDatabase)
   )
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1238330280

   @cloud-fan 
   
   Could you please take a look when you have some time?  This PR is a follow-up PR for https://github.com/apache/spark/pull/32364. Thanks!
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1239320853

   we should also update `V2SessionCatalog.defaultNamespace`


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r973074675


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -286,7 +284,7 @@ class SessionCatalog(
 
   def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
     val dbName = format(db)
-    if (dbName == DEFAULT_DATABASE) {
+    if (dbName == defaultDatabase) {

Review Comment:
   This is different. Let's say the current database is `abc` and the default database is `xyz`. Can we drop `xyz`?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r974976617


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -48,9 +48,6 @@ import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils}
 import org.apache.spark.util.Utils
 
-object SessionCatalog {
-  val DEFAULT_DATABASE = "default"

Review Comment:
   Yes, I have just reverted this part.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r977124693


##########
sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala:
##########
@@ -36,7 +36,7 @@ import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.FunctionIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
-import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE

Review Comment:
   I think we can revert all the changes in this file



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r978187565


##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,18 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if (SessionCatalog.DEFAULT_DATABASE != SQLConf.get.defaultDatabase) {
+        throw new SparkException(s"Default catalog database '${SQLConf.get.defaultDatabase}' " +
+          s"not exist, please create it first or change default database to 'default'.")

Review Comment:
   Here is a validation:
   
   ```
   $ bin/spark-shell --conf spark.sql.catalog.spark_catalog.defaultDatabase=other_db
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/23 04:00:12 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1663898413533).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 11.0.16)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   org.apache.spark.SparkDefaultCatalogDatabaseNotExistsException: [DEFAULT_CATALOG_DATABASE_NOT_EXISTS] Default catalog database other_db not exist, please create it first or change default database to 'default'.
     at org.apache.spark.sql.errors.QueryCompilationErrors$.defaultCatalogDatabaseNotExistsError(QueryCompilationErrors.scala:642)
     at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:156)
     at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:147)
     at org.apache.spark.sql.internal.BaseSessionStateBuilder.$anonfun$catalog$1(BaseSessionStateBuilder.scala:154)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog$lzycompute(SessionCatalog.scala:123)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog(SessionCatalog.scala:123)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.listDatabases(SessionCatalog.scala:324)
     at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.listNamespaces(V2SessionCatalog.scala:232)
     at org.apache.spark.sql.execution.datasources.v2.ShowNamespacesExec.run(ShowNamespacesExec.scala:42)```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][FOLLOWUP][Ranger][Hive][default db] Spark should not rely on the 'default' hive database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1228873858

   @cloud-fan, @hddong, @dongjoon-hyun could you please review my changes? Thanks!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] AmplabJenkins commented on pull request #37679: [SPARK-35242][FOLLOWUP][Ranger][Hive][default db] Spark should not rely on the 'default' hive database

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1229207971

   Can one of the admins verify this patch?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1259053324

   @cloud-fan,
   
   Thank you very much for your help!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r975364374


##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,18 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if ("default" != SQLConf.get.defaultDatabase) {

Review Comment:
   ```suggestion
         if (SessionCatalog.DEFAULT_DATABASE != SQLConf.get.defaultDatabase) {
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r975365487


##########
sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala:
##########
@@ -361,7 +360,7 @@ private[sql] trait SQLTestUtilsBase
 
     try f(dbName) finally {
       if (spark.catalog.currentDatabase == dbName) {
-        spark.sql(s"USE $DEFAULT_DATABASE")

Review Comment:
   It's test, I think we can still use `SessionCatalog.DEFAULT_DATABASE`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r978187565


##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,18 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if (SessionCatalog.DEFAULT_DATABASE != SQLConf.get.defaultDatabase) {
+        throw new SparkException(s"Default catalog database '${SQLConf.get.defaultDatabase}' " +
+          s"not exist, please create it first or change default database to 'default'.")

Review Comment:
   Here is a validation:
   
   ```
   $ bin/spark-shell --conf spark.sql.catalog.spark_catalog.defaultDatabase=other_db
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/23 02:38:25 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1663893506155).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 11.0.16)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   org.apache.spark.SparkDefaultCatalogDatabaseNotExistsException: [DEFAULT_CATALOG_DATABASE_NOT_EXISTS] Default catalog database other_db not exist, please create it first or change default database to 'default'.
     at org.apache.spark.sql.errors.QueryCompilationErrors$.defaultCatalogDatabaseNotExistsError(QueryCompilationErrors.scala:640)
     at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:153)
     at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:144)
     at org.apache.spark.sql.internal.BaseSessionStateBuilder.$anonfun$catalog$1(BaseSessionStateBuilder.scala:154)
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r978186198


##########
sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala:
##########
@@ -36,7 +36,7 @@ import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.FunctionIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
-import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE

Review Comment:
   Done



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r979213153


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -70,6 +70,11 @@
     ],
     "sqlState" : "22008"
   },
+  "DEFAULT_CATALOG_DATABASE_NOT_EXISTS" : {

Review Comment:
   Ok, I have renamed it.



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -70,6 +70,11 @@
     ],
     "sqlState" : "22008"
   },
+  "DEFAULT_CATALOG_DATABASE_NOT_EXISTS" : {
+    "message" : [
+      "Default catalog database <defaultDatabase> not exist, please create it first or change default database to 'default'. "

Review Comment:
   Done



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1250958346

   > Is this a common behavior in other databases?
   
   @cloud-fan Good question. The reason that we cannot delete the user specified default database because we have the following if statement in the actual code:
   
   ```
   if (dbName == defaultDatabase)
   ```
   
   and this is the latest state of master:
   
   https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala#L286
   
   ```
     def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
       val dbName = format(db)
       if (dbName == DEFAULT_DATABASE) {
         throw QueryCompilationErrors.cannotDropDefaultDatabaseError
       }
   ```
   
   As you can see that I am just using the same logic.
   
   If you think that we should only deny the database drop for "default" and allow for the value of spark.sql.catalog.spark_catalog.defaultDatabase, it is ok for me. The change is very simple:
   
   ```
   +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
   @@ -284,7 +284,7 @@ class SessionCatalog(
    
      def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
        val dbName = format(db)
   -    if (dbName == defaultDatabase) {
   +    if (dbName == "default") {
          throw QueryCompilationErrors.cannotDropDefaultDatabaseError
        }
        if (!ignoreIfNotExists) {
   ```
   
   and here is the validation:
   
   ```
    $ ./spark-shell --conf spark.sql.catalogImplementation=hive --conf spark.sql.catalog.spark_catalog.defaultDatabase=xyz
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/19 14:21:07 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1663590068068).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.17 (OpenJDK 64-Bit Server VM, Java 1.8.0_345)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   +---------+
   |namespace|
   +---------+
   |      abc|
   |  default|
   |      xyz|
   +---------+
   
   
   scala> spark.sql("use database abc")
   res1: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("SELECT current_database() AS db").show()
   +---+
   | db|
   +---+
   |abc|
   +---+
   
   
   scala> 
   
   scala> spark.sql("drop database xyz")
   res3: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("show databases").show()
   +---------+
   |namespace|
   +---------+
   |      abc|
   |  default|
   +---------+
   
   
   scala> 
   ```
   Is this solution acceptable for you?
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r973767185


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -286,7 +284,7 @@ class SessionCatalog(
 
   def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
     val dbName = format(db)
-    if (dbName == DEFAULT_DATABASE) {
+    if (dbName == defaultDatabase) {

Review Comment:
   @cloud-fan,
   
   If the default database is xyz and the current database is abc. We cannot drop the xyz database.
   
   Here are my validation steps::
   
   1)
   
   Test setup where the default database is the "default" and created the abc and the xyz databases:
   
   ```
   $ ./spark-shell --conf spark.sql.catalogImplementation=hive
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/18 21:23:12 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1663528992732).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 11.0.16)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("create database xyz")
   22/09/18 21:23:19 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
   22/09/18 21:23:19 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
   22/09/18 21:23:21 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
   22/09/18 21:23:21 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore roczei@192.168.0.38
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database default, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database xyz, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database xyz, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database global_temp, returning NoSuchObjectException
   22/09/18 21:23:21 WARN ObjectStore: Failed to get database xyz, returning NoSuchObjectException
   res0: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("create database abc")
   22/09/18 21:23:40 WARN ObjectStore: Failed to get database abc, returning NoSuchObjectException
   22/09/18 21:23:40 WARN ObjectStore: Failed to get database abc, returning NoSuchObjectException
   22/09/18 21:23:40 WARN ObjectStore: Failed to get database abc, returning NoSuchObjectException
   res1: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("show databases").show()
   +---------+
   |namespace|
   +---------+
   |      abc|
   |  default|
   |      xyz|
   +---------+
   
   scala> :quit
   ```
   
   
   2)
   
   ```
   $ ./spark-shell --conf spark.sql.catalogImplementation=hive --conf spark.sql.catalog.spark_catalog.defaultDatabase=xyz
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   22/09/18 21:24:05 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1663529046120).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 11.0.16)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   22/09/18 21:24:11 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
   22/09/18 21:24:11 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
   22/09/18 21:24:13 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
   22/09/18 21:24:13 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore roczei@192.168.0.38
   +---------+
   |namespace|
   +---------+
   |      abc|
   |  default|
   |      xyz|
   +---------+
   
   scala> spark.sql("use database abc")
   22/09/18 21:42:11 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
   22/09/18 21:42:11 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
   22/09/18 21:42:12 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
   22/09/18 21:42:12 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore roczei@192.168.0.38
   22/09/18 21:42:12 WARN ObjectStore: Failed to get database global_temp, returning NoSuchObjectException
   res0: org.apache.spark.sql.DataFrame = []
   
   scala> spark.sql("SELECT current_database() AS db").show()
   +---+
   | db|
   +---+
   |abc|
   +---+
   
   
   scala> spark.sql("drop database xyz")
   org.apache.spark.sql.AnalysisException: Can not drop default database
     at org.apache.spark.sql.errors.QueryCompilationErrors$.cannotDropDefaultDatabaseError(QueryCompilationErrors.scala:635)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.dropDatabase(SessionCatalog.scala:288)
     at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.dropNamespace(V2SessionCatalog.scala:300)
     at org.apache.spark.sql.execution.datasources.v2.DropNamespaceExec.run(DropNamespaceExec.scala:42)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:111)
     at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:171)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
     at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
     at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:97)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r975362828


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala:
##########
@@ -130,7 +130,7 @@ class CatalogManager(
       _currentNamespace = None
       // Reset the current database of v1 `SessionCatalog` when switching current catalog, so that
       // when we switch back to session catalog, the current namespace definitely is ["default"].
-      v1SessionCatalog.setCurrentDatabase(SessionCatalog.DEFAULT_DATABASE)
+      v1SessionCatalog.setCurrentDatabase(conf.getConf(StaticSQLConf.CATALOG_DEFAULT_DATABASE))

Review Comment:
   ```suggestion
         v1SessionCatalog.setCurrentDatabase(conf.defaultDatabase)
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala:
##########
@@ -144,7 +144,7 @@ class CatalogManager(
     catalogs.clear()
     _currentNamespace = None
     _currentCatalogName = None
-    v1SessionCatalog.setCurrentDatabase(SessionCatalog.DEFAULT_DATABASE)
+    v1SessionCatalog.setCurrentDatabase(conf.getConf(StaticSQLConf.CATALOG_DEFAULT_DATABASE))

Review Comment:
   ditto



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r975361460


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1846,23 +1848,23 @@ class SessionCatalog(
   // -----------------
 
   /**
-   * Drop all existing databases (except "default"), tables, partitions and functions,
-   * and set the current database to "default".
+   * Drop all existing databases (except defaultDatabase), tables, partitions and functions,
+   * and set the current database to defaultDatabase.
    *
    * This is mainly used for tests.
    */
   def reset(): Unit = synchronized {
-    setCurrentDatabase(DEFAULT_DATABASE)
-    externalCatalog.setCurrentDatabase(DEFAULT_DATABASE)
-    listDatabases().filter(_ != DEFAULT_DATABASE).foreach { db =>
+    setCurrentDatabase(defaultDatabase)
+    externalCatalog.setCurrentDatabase(defaultDatabase)
+    listDatabases().filter(_ != defaultDatabase).foreach { db =>

Review Comment:
   We can't drop DEFAULT database, so we should still filter by `DEFAULT_DATABASE` here.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1253877919

   Hi @cloud-fan,
   
   Thanks for the feedbacks, I have resolved all of them. Unfortunatelly I did a mistake during the version magament but I have already resolved with git rebase / git push as you can see.  This is the reason that a lot of new labels have been added which are not necessary by this pull request. Only the SQL label is the good one.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r979572820


##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,19 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if (SessionCatalog.DEFAULT_DATABASE != SQLConf.get.defaultDatabase) {

Review Comment:
   ```suggestion
         if (SessionCatalog.DEFAULT_DATABASE.equalsIgnoreCase(SQLConf.get.defaultDatabase)) {
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r967420952


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -286,7 +284,7 @@ class SessionCatalog(
 
   def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
     val dbName = format(db)
-    if (dbName == DEFAULT_DATABASE) {
+    if (dbName == defaultDatabase) {

Review Comment:
   @cloud-fan, 
   
   Here is an example where the user specified default database is "other". Currently if we try to drop "other", it will fail with "Can not drop default database" message. 
   
   ```
   $ bin/spark-shell --conf spark.sql.catalogImplementation=hive --conf spark.sql.catalog.spark_catalog.defaultDatabase=other
   Setting default log level to "WARN".
   To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   Spark context Web UI available at http://localhost:4040
   Spark context available as 'sc' (master = local[*], app id = local-1662751254354).
   Spark session available as 'spark'.
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
         /_/
            
   Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 1.8.0_345)
   Type in expressions to have them evaluated.
   Type :help for more information.
   
   scala> spark.sql("show databases").show()
   +---------+
   |namespace|
   +---------+
   |  default|
   |    other|
   +---------+
   
   
   scala> spark.sql("drop database other")
   org.apache.spark.sql.AnalysisException: Can not drop default database
     at org.apache.spark.sql.errors.QueryCompilationErrors$.cannotDropDefaultDatabaseError(QueryCompilationErrors.scala:635)
     at org.apache.spark.sql.catalyst.catalog.SessionCatalog.dropDatabase(SessionCatalog.scala:288)
     at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.dropNamespace(V2SessionCatalog.scala:299)
     at org.apache.spark.sql.execution.datasources.v2.DropNamespaceExec.run(DropNamespaceExec.scala:42)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
     at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:111)
     at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:171)
     at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
     at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
     at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
     at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:97)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:505)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
   ```
   
   I am not sure that I fully understand the request here. Would you like to allow this "drop database other"? Please elaborate a bit.
   
   There was a previous Spark jira to prevent dropping current database, so probably we should keep this. Related jira:
   
   Prevent dropping current database: 
   
   - https://issues.apache.org/jira/browse/SPARK-16459
   - https://github.com/apache/spark/pull/14115



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r964775576


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -286,7 +284,7 @@ class SessionCatalog(
 
   def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = {
     val dbName = format(db)
-    if (dbName == DEFAULT_DATABASE) {
+    if (dbName == defaultDatabase) {

Review Comment:
   We can't drop the `DEFAULT` database, but it doesn't mean we can't drop the user-specified default current database (can be `abc` or something).



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r975361715


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1846,23 +1848,23 @@ class SessionCatalog(
   // -----------------
 
   /**
-   * Drop all existing databases (except "default"), tables, partitions and functions,
-   * and set the current database to "default".
+   * Drop all existing databases (except defaultDatabase), tables, partitions and functions,
+   * and set the current database to defaultDatabase.
    *
    * This is mainly used for tests.
    */
   def reset(): Unit = synchronized {
-    setCurrentDatabase(DEFAULT_DATABASE)
-    externalCatalog.setCurrentDatabase(DEFAULT_DATABASE)
-    listDatabases().filter(_ != DEFAULT_DATABASE).foreach { db =>
+    setCurrentDatabase(defaultDatabase)
+    externalCatalog.setCurrentDatabase(defaultDatabase)
+    listDatabases().filter(_ != defaultDatabase).foreach { db =>
       dropDatabase(db, ignoreIfNotExists = false, cascade = true)
     }
-    listTables(DEFAULT_DATABASE).foreach { table =>
+    listTables(defaultDatabase).foreach { table =>

Review Comment:
   ditto



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1846,23 +1848,23 @@ class SessionCatalog(
   // -----------------
 
   /**
-   * Drop all existing databases (except "default"), tables, partitions and functions,
-   * and set the current database to "default".
+   * Drop all existing databases (except defaultDatabase), tables, partitions and functions,
+   * and set the current database to defaultDatabase.
    *
    * This is mainly used for tests.
    */
   def reset(): Unit = synchronized {
-    setCurrentDatabase(DEFAULT_DATABASE)
-    externalCatalog.setCurrentDatabase(DEFAULT_DATABASE)
-    listDatabases().filter(_ != DEFAULT_DATABASE).foreach { db =>
+    setCurrentDatabase(defaultDatabase)
+    externalCatalog.setCurrentDatabase(defaultDatabase)
+    listDatabases().filter(_ != defaultDatabase).foreach { db =>
       dropDatabase(db, ignoreIfNotExists = false, cascade = true)
     }
-    listTables(DEFAULT_DATABASE).foreach { table =>
+    listTables(defaultDatabase).foreach { table =>
       dropTable(table, ignoreIfNotExists = false, purge = false)
     }
     // Temp functions are dropped below, we only need to drop permanent functions here.
-    externalCatalog.listFunctions(DEFAULT_DATABASE, "*").map { f =>
-      FunctionIdentifier(f, Some(DEFAULT_DATABASE))
+    externalCatalog.listFunctions(defaultDatabase, "*").map { f =>

Review Comment:
   ditto



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r975364879


##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,18 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if ("default" != SQLConf.get.defaultDatabase) {
+        throw new SparkException(s"Default catalog database '${SQLConf.get.defaultDatabase}' " +
+          s"not exist, please change default database to 'default' and create it first.")

Review Comment:
   ```suggestion
             s"not exist, please create it first or change default database to 'default'.")
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r978389370


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -70,6 +70,11 @@
     ],
     "sqlState" : "22008"
   },
+  "DEFAULT_CATALOG_DATABASE_NOT_EXISTS" : {

Review Comment:
   ```suggestion
     "DEFAULT_DATABASE_NOT_EXISTS" : {
   ```



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -70,6 +70,11 @@
     ],
     "sqlState" : "22008"
   },
+  "DEFAULT_CATALOG_DATABASE_NOT_EXISTS" : {
+    "message" : [
+      "Default catalog database <defaultDatabase> not exist, please create it first or change default database to 'default'. "

Review Comment:
   ```suggestion
         "Default database <defaultDatabase> not exist, please create it first or change default database to 'default'. "
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] cloud-fan commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1258818080

   thanks, meriging to master!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r979762248


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1932,6 +1932,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
     new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast")
   }
 
+  def defaultDatabaseNotExistsError(defaultDatabase: String): Throwable = {
+    new SparkDefaultDatabaseNotExistsException(

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,19 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if (SessionCatalog.DEFAULT_DATABASE != SQLConf.get.defaultDatabase) {

Review Comment:
   Done



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on PR #37679:
URL: https://github.com/apache/spark/pull/37679#issuecomment-1257736748

   > LGTM except for 2 minor comments
   
   Thanks @cloud-fan! I have implemented these two comments.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database

Posted by GitBox <gi...@apache.org>.
roczei commented on code in PR #37679:
URL: https://github.com/apache/spark/pull/37679#discussion_r978186734


##########
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala:
##########
@@ -148,13 +148,18 @@ private[sql] class SharedState(
     val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
       SharedState.externalCatalogClassName(conf), conf, hadoopConf)
 
-    val defaultDbDefinition = CatalogDatabase(
-      SessionCatalog.DEFAULT_DATABASE,
-      "default database",
-      CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)),
-      Map())
     // Create default database if it doesn't exist
-    if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) {
+    // If database name not equals 'default', throw exception
+    if (!externalCatalog.databaseExists(SQLConf.get.defaultDatabase)) {
+      if (SessionCatalog.DEFAULT_DATABASE != SQLConf.get.defaultDatabase) {
+        throw new SparkException(s"Default catalog database '${SQLConf.get.defaultDatabase}' " +
+          s"not exist, please create it first or change default database to 'default'.")

Review Comment:
   @cloud-fan,
    
   I have added it to error-classes.json, please verify.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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