[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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 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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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| +-+
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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| +-+
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] [spark] roczei commented on a diff in pull request #37679: [SPARK-35242][SQL] Support changing session catalog's default database
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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org