diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md index cd8f5b770ae..1ba684f2799 100644 --- a/docs/deployment/settings.md +++ b/docs/deployment/settings.md @@ -329,23 +329,24 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co ### Operation -| Key | Default | Meaning | Type | Since | -|-------------------------------------------------|---------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|-------| -| kyuubi.operation.idle.timeout | PT3H | Operation will be closed when it's not accessed for this duration of time | duration | 1.0.0 | -| kyuubi.operation.interrupt.on.cancel | true | When true, all running tasks will be interrupted if one cancels a query. When false, all running tasks will remain until finished. | boolean | 1.2.0 | -| kyuubi.operation.language | SQL | Choose a programing language for the following inputs | string | 1.5.0 | -| kyuubi.operation.log.dir.root | server_operation_logs | Root directory for query operation log at server-side. | string | 1.4.0 | -| kyuubi.operation.plan.only.excludes | ResetCommand,SetCommand,SetNamespaceCommand,UseStatement,SetCatalogAndNamespace | Comma-separated list of query plan names, in the form of simple class names, i.e, for `SET abc=xyz`, the value will be `SetCommand`. For those auxiliary plans, such as `switch databases`, `set properties`, or `create temporary view` etc., which are used for setup evaluating environments for analyzing actual queries, we can use this config to exclude them and let them take effect. See also kyuubi.operation.plan.only.mode. | seq | 1.5.0 | -| kyuubi.operation.plan.only.mode | none | Configures the statement performed mode, The value can be 'parse', 'analyze', 'optimize', 'optimize_with_stats', 'physical', 'execution', or 'none', when it is 'none', indicate to the statement will be fully executed, otherwise only way without executing the query. different engines currently support different modes, the Spark engine supports all modes, and the Flink engine supports 'parse', 'physical', and 'execution', other engines do not support planOnly currently. | string | 1.4.0 | -| kyuubi.operation.plan.only.output.style | plain | Configures the planOnly output style. The value can be 'plain' or 'json', and the default value is 'plain'. This configuration supports only the output styles of the Spark engine | string | 1.7.0 | -| kyuubi.operation.progress.enabled | false | Whether to enable the operation progress. When true, the operation progress will be returned in `GetOperationStatus`. | boolean | 1.6.0 | -| kyuubi.operation.query.timeout | <undefined> | Timeout for query executions at server-side, take effect with client-side timeout(`java.sql.Statement.setQueryTimeout`) together, a running query will be cancelled automatically if timeout. It's off by default, which means only client-side take full control of whether the query should timeout or not. If set, client-side timeout is capped at this point. To cancel the queries right away without waiting for task to finish, consider enabling kyuubi.operation.interrupt.on.cancel together. | duration | 1.2.0 | -| kyuubi.operation.result.arrow.timestampAsString | false | When true, arrow-based rowsets will convert columns of type timestamp to strings for transmission. | boolean | 1.7.0 | -| kyuubi.operation.result.format | thrift | Specify the result format, available configs are: | string | 1.7.0 | -| kyuubi.operation.result.max.rows | 0 | Max rows of Spark query results. Rows exceeding the limit would be ignored. By setting this value to 0 to disable the max rows limit. | int | 1.6.0 | -| kyuubi.operation.scheduler.pool | <undefined> | The scheduler pool of job. Note that, this config should be used after changing Spark config spark.scheduler.mode=FAIR. | string | 1.1.1 | -| kyuubi.operation.spark.listener.enabled | true | When set to true, Spark engine registers an SQLOperationListener before executing the statement, logging a few summary statistics when each stage completes. | boolean | 1.6.0 | -| kyuubi.operation.status.polling.timeout | PT5S | Timeout(ms) for long polling asynchronous running sql query's status | duration | 1.0.0 | +| Key | Default | Meaning | Type | Since | +|--------------------------------------------------|---------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|-------| +| kyuubi.operation.getTables.ignoreTableProperties | false | Speed up the `GetTables` operation by returning table identities only. | boolean | 1.8.0 | +| kyuubi.operation.idle.timeout | PT3H | Operation will be closed when it's not accessed for this duration of time | duration | 1.0.0 | +| kyuubi.operation.interrupt.on.cancel | true | When true, all running tasks will be interrupted if one cancels a query. When false, all running tasks will remain until finished. | boolean | 1.2.0 | +| kyuubi.operation.language | SQL | Choose a programing language for the following inputs | string | 1.5.0 | +| kyuubi.operation.log.dir.root | server_operation_logs | Root directory for query operation log at server-side. | string | 1.4.0 | +| kyuubi.operation.plan.only.excludes | ResetCommand,SetCommand,SetNamespaceCommand,UseStatement,SetCatalogAndNamespace | Comma-separated list of query plan names, in the form of simple class names, i.e, for `SET abc=xyz`, the value will be `SetCommand`. For those auxiliary plans, such as `switch databases`, `set properties`, or `create temporary view` etc., which are used for setup evaluating environments for analyzing actual queries, we can use this config to exclude them and let them take effect. See also kyuubi.operation.plan.only.mode. | seq | 1.5.0 | +| kyuubi.operation.plan.only.mode | none | Configures the statement performed mode, The value can be 'parse', 'analyze', 'optimize', 'optimize_with_stats', 'physical', 'execution', or 'none', when it is 'none', indicate to the statement will be fully executed, otherwise only way without executing the query. different engines currently support different modes, the Spark engine supports all modes, and the Flink engine supports 'parse', 'physical', and 'execution', other engines do not support planOnly currently. | string | 1.4.0 | +| kyuubi.operation.plan.only.output.style | plain | Configures the planOnly output style. The value can be 'plain' or 'json', and the default value is 'plain'. This configuration supports only the output styles of the Spark engine | string | 1.7.0 | +| kyuubi.operation.progress.enabled | false | Whether to enable the operation progress. When true, the operation progress will be returned in `GetOperationStatus`. | boolean | 1.6.0 | +| kyuubi.operation.query.timeout | <undefined> | Timeout for query executions at server-side, take effect with client-side timeout(`java.sql.Statement.setQueryTimeout`) together, a running query will be cancelled automatically if timeout. It's off by default, which means only client-side take full control of whether the query should timeout or not. If set, client-side timeout is capped at this point. To cancel the queries right away without waiting for task to finish, consider enabling kyuubi.operation.interrupt.on.cancel together. | duration | 1.2.0 | +| kyuubi.operation.result.arrow.timestampAsString | false | When true, arrow-based rowsets will convert columns of type timestamp to strings for transmission. | boolean | 1.7.0 | +| kyuubi.operation.result.format | thrift | Specify the result format, available configs are: | string | 1.7.0 | +| kyuubi.operation.result.max.rows | 0 | Max rows of Spark query results. Rows exceeding the limit would be ignored. By setting this value to 0 to disable the max rows limit. | int | 1.6.0 | +| kyuubi.operation.scheduler.pool | <undefined> | The scheduler pool of job. Note that, this config should be used after changing Spark config spark.scheduler.mode=FAIR. | string | 1.1.1 | +| kyuubi.operation.spark.listener.enabled | true | When set to true, Spark engine registers an SQLOperationListener before executing the statement, logging a few summary statistics when each stage completes. | boolean | 1.6.0 | +| kyuubi.operation.status.polling.timeout | PT5S | Timeout(ms) for long polling asynchronous running sql query's status | duration | 1.0.0 | ### Server diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/GetTables.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/GetTables.scala index 4093c61c100..40642b825b9 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/GetTables.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/GetTables.scala @@ -19,6 +19,7 @@ package org.apache.kyuubi.engine.spark.operation import org.apache.spark.sql.types.StructType +import org.apache.kyuubi.config.KyuubiConf.OPERATION_GET_TABLES_IGNORE_TABLE_PROPERTIES import org.apache.kyuubi.engine.spark.shim.SparkCatalogShim import org.apache.kyuubi.operation.IterableFetchIterator import org.apache.kyuubi.operation.meta.ResultSetSchemaConstant._ @@ -32,6 +33,12 @@ class GetTables( tableTypes: Set[String]) extends SparkOperation(session) { + protected val ignoreTableProperties = + spark.conf.getOption(OPERATION_GET_TABLES_IGNORE_TABLE_PROPERTIES.key) match { + case Some(s) => s.toBoolean + case _ => session.sessionManager.getConf.get(OPERATION_GET_TABLES_IGNORE_TABLE_PROPERTIES) + } + override def statement: String = { super.statement + s" [catalog: $catalog," + @@ -68,7 +75,13 @@ class GetTables( val tablePattern = toJavaRegex(tableName) val sparkShim = SparkCatalogShim() val catalogTablesAndViews = - sparkShim.getCatalogTablesOrViews(spark, catalog, schemaPattern, tablePattern, tableTypes) + sparkShim.getCatalogTablesOrViews( + spark, + catalog, + schemaPattern, + tablePattern, + tableTypes, + ignoreTableProperties) val allTableAndViews = if (tableTypes.exists("VIEW".equalsIgnoreCase)) { diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v2_4.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v2_4.scala index 0f6195acf3f..ea72dd1563c 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v2_4.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v2_4.scala @@ -64,7 +64,8 @@ class CatalogShim_v2_4 extends SparkCatalogShim { catalogName: String, schemaPattern: String, tablePattern: String, - tableTypes: Set[String]): Seq[Row] = { + tableTypes: Set[String], + ignoreTableProperties: Boolean): Seq[Row] = { val catalog = spark.sessionState.catalog val databases = catalog.listDatabases(schemaPattern) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v3_0.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v3_0.scala index a663ba63638..27c524f3032 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v3_0.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/CatalogShim_v3_0.scala @@ -150,7 +150,8 @@ class CatalogShim_v3_0 extends CatalogShim_v2_4 { catalogName: String, schemaPattern: String, tablePattern: String, - tableTypes: Set[String]): Seq[Row] = { + tableTypes: Set[String], + ignoreTableProperties: Boolean = false): Seq[Row] = { val catalog = getCatalog(spark, catalogName) val namespaces = listNamespacesWithPattern(catalog, schemaPattern) catalog match { @@ -160,16 +161,17 @@ class CatalogShim_v3_0 extends CatalogShim_v2_4 { SESSION_CATALOG, schemaPattern, tablePattern, - tableTypes) + tableTypes, + ignoreTableProperties) case tc: TableCatalog => val tp = tablePattern.r.pattern val identifiers = namespaces.flatMap { ns => tc.listTables(ns).filter(i => tp.matcher(quoteIfNeeded(i.name())).matches()) } identifiers.map { ident => - val table = tc.loadTable(ident) // TODO: restore view type for session catalog - val comment = table.properties().getOrDefault(TableCatalog.PROP_COMMENT, "") + val comment = if (ignoreTableProperties) "" + else tc.loadTable(ident).properties().getOrDefault(TableCatalog.PROP_COMMENT, "") val schema = ident.namespace().map(quoteIfNeeded).mkString(".") val tableName = quoteIfNeeded(ident.name()) Row(catalog.name(), schema, tableName, "TABLE", comment, null, null, null, null, null) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/SparkCatalogShim.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/SparkCatalogShim.scala index bc5792823f7..83c80652380 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/SparkCatalogShim.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/shim/SparkCatalogShim.scala @@ -69,7 +69,8 @@ trait SparkCatalogShim extends Logging { catalogName: String, schemaPattern: String, tablePattern: String, - tableTypes: Set[String]): Seq[Row] + tableTypes: Set[String], + ignoreTableProperties: Boolean): Seq[Row] def getTempViews( spark: SparkSession, diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala index f61cfeaa756..c836b016575 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala @@ -2713,4 +2713,11 @@ object KyuubiConf { .version("1.7.0") .timeConf .createWithDefault(Duration.ofSeconds(60).toMillis) + + val OPERATION_GET_TABLES_IGNORE_TABLE_PROPERTIES: ConfigEntry[Boolean] = + buildConf("kyuubi.operation.getTables.ignoreTableProperties") + .doc("Speed up the `GetTables` operation by returning table identities only.") + .version("1.8.0") + .booleanConf + .createWithDefault(false) }