diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index d547e6d326398..a77f8619b7b36 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -7444,6 +7444,11 @@ "The ANALYZE TABLE command does not support views." ] }, + "CATALOG_INTERFACE_METHOD" : { + "message" : [ + "Catalog API is not supported by ." + ] + }, "CATALOG_OPERATION" : { "message" : [ "Catalog does not support ." diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md index 16bce4527fdab..f6d130d34d3de 100644 --- a/docs/sql-performance-tuning.md +++ b/docs/sql-performance-tuning.md @@ -30,6 +30,8 @@ Spark SQL can cache tables using an in-memory columnar format by calling `spark. Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` or `dataFrame.unpersist()` to remove the table from memory. +To list relations cached with an explicit name, use `SHOW CACHED TABLES` in SQL or `spark.catalog.listCachedTables()`. Entries cached only via `Dataset.cache()` without a name are not included. + Configuration of in-memory caching can be done via `spark.conf.set` or by running `SET key=value` commands using SQL. diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 78b44c97a4976..f8181be1a351c 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -440,6 +440,7 @@ Below is a list of all the keywords in Spark SQL. |BY|non-reserved|non-reserved|reserved| |BYTE|non-reserved|non-reserved|non-reserved| |CACHE|non-reserved|non-reserved|non-reserved| +|CACHED|non-reserved|non-reserved|non-reserved| |CALL|reserved|non-reserved|reserved| |CALLED|non-reserved|non-reserved|non-reserved| |CASCADE|non-reserved|non-reserved|non-reserved| diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md index ae9e208e7f4e0..69cfb3c133dec 100644 --- a/docs/sql-ref-syntax-aux-cache-cache-table.md +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -79,6 +79,7 @@ CACHE TABLE testCache OPTIONS ('storageLevel' 'DISK_ONLY') SELECT * FROM testDat ### Related Statements +* [SHOW CACHED TABLES](sql-ref-syntax-aux-show-cached-tables.html) * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index b8ae8e3d4cefc..5dcf95e63f43e 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -49,6 +49,7 @@ UNCACHE TABLE t1; ### Related Statements * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) +* [SHOW CACHED TABLES](sql-ref-syntax-aux-show-cached-tables.html) * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-show-cached-tables.md b/docs/sql-ref-syntax-aux-show-cached-tables.md new file mode 100644 index 0000000000000..90855e9584a9e --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-cached-tables.md @@ -0,0 +1,51 @@ +--- +layout: global +title: SHOW CACHED TABLES +displayTitle: SHOW CACHED TABLES +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description + +The `SHOW CACHED TABLES` statement returns every in-memory cache entry that was registered with an explicit table or view name, for example via [`CACHE TABLE`](sql-ref-syntax-aux-cache-cache-table.html) or `spark.catalog.cacheTable`. The result has two columns: `tableName` (the name used when caching) and `storageLevel` (a string description of how the data is cached). + +Relations cached only through `Dataset.cache()` / `DataFrame.cache()` without assigning a catalog name are **not** listed. + +### Syntax + +```sql +SHOW CACHED TABLES +``` + +### Examples + +```sql +CACHE TABLE my_table AS SELECT * FROM src; + +SHOW CACHED TABLES; ++----------+--------------------------------------+ +| tableName| storageLevel| ++----------+--------------------------------------+ +| my_table|Disk Memory Deserialized 1x Replicated| ++----------+--------------------------------------+ +``` + +### Related Statements + +* [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) +* [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) +* [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) diff --git a/docs/sql-ref-syntax-aux-show.md b/docs/sql-ref-syntax-aux-show.md index 9f64ea2d50ae1..ecca4d81746ac 100644 --- a/docs/sql-ref-syntax-aux-show.md +++ b/docs/sql-ref-syntax-aux-show.md @@ -19,6 +19,7 @@ license: | limitations under the License. --- + * [SHOW CACHED TABLES](sql-ref-syntax-aux-show-cached-tables.html) * [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) * [SHOW CREATE TABLE](sql-ref-syntax-aux-show-create-table.html) * [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index 0a2b9ba34b522..cf33dd5efb2e1 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -124,6 +124,7 @@ You use SQL scripting to execute procedural logic in SQL. * [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) * [SET VAR](sql-ref-syntax-aux-set-var.html) + * [SHOW CACHED TABLES](sql-ref-syntax-aux-show-cached-tables.html) * [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) * [SHOW CREATE TABLE](sql-ref-syntax-aux-show-create-table.html) * [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 89d1474190387..599dab36e5dbb 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -35,6 +35,10 @@ object MimaExcludes { // Exclude rules for 4.2.x from 4.1.0 lazy val v42excludes = v41excludes ++ Seq( + // [SQL] SafeJsonSerializer.safeMapToJValue: second parameter widened from Function1 to + // Function2 so the key is passed to the value serializer (progress.scala). Binary-incompatible + // vs spark-sql-api 4.0.0; not part of the public supported API (private[streaming] package). + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.SafeJsonSerializer.safeMapToJValue"), // Add DEBUG format to ErrorMessageFormat enum ProblemFilters.exclude[Problem]("org.apache.spark.ErrorMessageFormat*"), // [SPARK-47086][BUILD][CORE][WEBUI] Upgrade Jetty to 12.1.4 diff --git a/python/docs/source/reference/pyspark.sql/catalog.rst b/python/docs/source/reference/pyspark.sql/catalog.rst index 742af104dfba8..4e9da6f41fe30 100644 --- a/python/docs/source/reference/pyspark.sql/catalog.rst +++ b/python/docs/source/reference/pyspark.sql/catalog.rst @@ -25,25 +25,35 @@ Catalog .. autosummary:: :toctree: api/ + Catalog.analyzeTable Catalog.cacheTable Catalog.clearCache + Catalog.createDatabase Catalog.createExternalTable Catalog.createTable Catalog.currentCatalog Catalog.currentDatabase Catalog.databaseExists + Catalog.dropDatabase Catalog.dropGlobalTempView + Catalog.dropTable Catalog.dropTempView + Catalog.dropView Catalog.functionExists + Catalog.getCreateTableString Catalog.getDatabase Catalog.getFunction Catalog.getTable + Catalog.getTableProperties Catalog.isCached + Catalog.listCachedTables Catalog.listCatalogs Catalog.listColumns Catalog.listDatabases Catalog.listFunctions + Catalog.listPartitions Catalog.listTables + Catalog.listViews Catalog.recoverPartitions Catalog.refreshByPath Catalog.refreshTable @@ -51,4 +61,5 @@ Catalog Catalog.setCurrentCatalog Catalog.setCurrentDatabase Catalog.tableExists + Catalog.truncateTable Catalog.uncacheTable diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 268f4e0a05227..f3123fa54f261 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -17,7 +17,7 @@ import sys import warnings -from typing import Any, Callable, NamedTuple, List, Optional, TYPE_CHECKING +from typing import Any, Callable, Dict, NamedTuple, List, Optional, TYPE_CHECKING from pyspark.errors import PySparkTypeError from pyspark.storagelevel import StorageLevel @@ -77,10 +77,20 @@ class Function(NamedTuple): isTemporary: bool +class CachedTable(NamedTuple): + name: str + storageLevel: str + + +class CatalogTablePartition(NamedTuple): + partition: str + + class Catalog: - """User-facing catalog API, accessible through `SparkSession.catalog`. + """Spark SQL catalog interface. - This is a thin wrapper around its Scala implementation org.apache.spark.sql.catalog.Catalog. + Use :attr:`~pyspark.sql.SparkSession.catalog` on an active session. This class is a thin + wrapper around ``org.apache.spark.sql.catalog.Catalog``. .. versionchanged:: 3.4.0 Supports Spark Connect. @@ -94,10 +104,15 @@ def __init__(self, sparkSession: SparkSession) -> None: self._jcatalog = sparkSession._jsparkSession.catalog() def currentCatalog(self) -> str: - """Returns the current default catalog in this session. + """Returns the current catalog in this session. .. versionadded:: 3.4.0 + Returns + ------- + str + The current catalog name. + Examples -------- >>> spark.catalog.currentCatalog() @@ -106,14 +121,14 @@ def currentCatalog(self) -> str: return self._jcatalog.currentCatalog() def setCurrentCatalog(self, catalogName: str) -> None: - """Sets the current default catalog in this session. + """Sets the current catalog in this session. .. versionadded:: 3.4.0 Parameters ---------- catalogName : str - name of the catalog to set + Name of the catalog to set. Examples -------- @@ -122,16 +137,18 @@ def setCurrentCatalog(self, catalogName: str) -> None: return self._jcatalog.setCurrentCatalog(catalogName) def listCatalogs(self, pattern: Optional[str] = None) -> List[CatalogMetadata]: - """Returns a list of catalogs in this session. + """Returns a list of catalogs available in this session. + + With ``pattern``, returns only catalogs whose name matches that pattern. .. versionadded:: 3.4.0 Parameters ---------- pattern : str, optional - The pattern that the catalog name needs to match. + Pattern that catalog names must match. - .. versionadded: 3.5.0 + .. versionadded:: 3.5.0 Returns ------- @@ -161,16 +178,361 @@ def listCatalogs(self, pattern: Optional[str] = None) -> List[CatalogMetadata]: ) return catalogs + def listCachedTables(self) -> List[CachedTable]: + """Lists named in-memory cache entries (same as ``SHOW CACHED TABLES``). + + Includes caches registered with ``CACHE TABLE`` or + :meth:`~pyspark.sql.catalog.Catalog.cacheTable`. Caches from + :meth:`~pyspark.sql.DataFrame.cache` without a name are not listed. + + .. versionadded:: 4.2.0 + + Returns + ------- + list + A list of :class:`CachedTable` describing each named cache entry. + + Examples + -------- + >>> _ = spark.sql("DROP TABLE IF EXISTS tbl_cached_list") + >>> _ = spark.sql("CREATE TABLE tbl_cached_list (id INT) USING parquet") + >>> spark.catalog.clearCache() + >>> spark.catalog.listCachedTables() + [] + >>> spark.catalog.cacheTable("tbl_cached_list") + >>> any("tbl_cached_list" in ct.name for ct in spark.catalog.listCachedTables()) + True + >>> spark.catalog.uncacheTable("tbl_cached_list") + >>> _ = spark.sql("DROP TABLE tbl_cached_list") + """ + iter = self._jcatalog.listCachedTables().toLocalIterator() + out: List[CachedTable] = [] + while iter.hasNext(): + j = iter.next() + out.append(CachedTable(name=j.name(), storageLevel=j.storageLevel())) + return out + + def dropTable(self, tableName: str, ifExists: bool = False, purge: bool = False) -> None: + """Drops a persistent table. + + This does not remove temp views; use :meth:`Catalog.dropTempView`. + + .. versionadded:: 4.2.0 + + Parameters + ---------- + tableName : str + Name of the table to drop. May be qualified with catalog and database (namespace). + ifExists : bool, optional + If ``True``, do not fail when the table does not exist. + purge : bool, optional + If ``True``, skip moving data to a trash directory when the catalog supports it. + + Examples + -------- + >>> _ = spark.sql("DROP TABLE IF EXISTS tbl_drop_doc") + >>> _ = spark.sql("CREATE TABLE tbl_drop_doc (id INT) USING parquet") + >>> spark.catalog.dropTable("tbl_drop_doc") + >>> spark.catalog.tableExists("tbl_drop_doc") + False + """ + self._jcatalog.dropTable(tableName, ifExists, purge) + + def dropView(self, viewName: str, ifExists: bool = False) -> None: + """Drops a persistent view. + + .. versionadded:: 4.2.0 + + Parameters + ---------- + viewName : str + Name of the view to drop. May be qualified with catalog and database (namespace). + ifExists : bool, optional + If ``True``, do not fail when the view does not exist. + + Examples + -------- + >>> _ = spark.sql("DROP VIEW IF EXISTS view_drop_doc") + >>> _ = spark.sql("CREATE VIEW view_drop_doc AS SELECT 1 AS c") + >>> spark.catalog.dropView("view_drop_doc") + >>> spark.catalog.tableExists("view_drop_doc") + False + """ + self._jcatalog.dropView(viewName, ifExists) + + def createDatabase( + self, dbName: str, ifNotExists: bool = False, properties: Optional[Dict[str, str]] = None + ) -> None: + """Creates a namespace (database/schema). + + ``dbName`` may be a multi-part identifier (for example ``catalog.schema``). Optional + ``properties`` follow ``CREATE NAMESPACE`` (for example comment or location keys). + + .. versionadded:: 4.2.0 + + Parameters + ---------- + dbName : str + Name of the namespace to create. + ifNotExists : bool, optional + If ``True``, do not fail when the namespace already exists. + properties : dict, optional + Map of namespace property keys to string values. + + Examples + -------- + >>> spark.catalog.dropDatabase("db_create_doc", ifExists=True, cascade=True) + >>> spark.catalog.createDatabase("db_create_doc") + >>> spark.catalog.databaseExists("db_create_doc") + True + >>> spark.catalog.dropDatabase("db_create_doc", cascade=True) + >>> spark.catalog.databaseExists("db_create_doc") + False + """ + sc = self._sc + assert sc is not None + ju = sc._gateway.jvm.java.util # type: ignore[union-attr] + m = ju.HashMap() + if properties: + for k, v in properties.items(): + m.put(k, v) + self._jcatalog.createDatabase(dbName, ifNotExists, m) + + def dropDatabase(self, dbName: str, ifExists: bool = False, cascade: bool = False) -> None: + """Drops a namespace. + + .. versionadded:: 4.2.0 + + Parameters + ---------- + dbName : str + Name of the namespace to drop. May be qualified with catalog name. + ifExists : bool, optional + If ``True``, do not fail when the namespace does not exist. + cascade : bool, optional + If ``True``, also drop tables and functions in the namespace. + + Examples + -------- + >>> spark.catalog.dropDatabase("db_drop_doc", ifExists=True, cascade=True) + >>> spark.catalog.createDatabase("db_drop_doc") + >>> spark.catalog.dropDatabase("db_drop_doc", cascade=True) + >>> spark.catalog.databaseExists("db_drop_doc") + False + """ + self._jcatalog.dropDatabase(dbName, ifExists, cascade) + + def listPartitions(self, tableName: str) -> List[CatalogTablePartition]: + """Lists partition value strings for a table (same as ``SHOW PARTITIONS``). + + .. versionadded:: 4.2.0 + + Parameters + ---------- + tableName : str + Name of the partitioned table. May be qualified with catalog and database (namespace). + + Returns + ------- + list + A list of :class:`CatalogTablePartition` (each ``partition`` field is a spec string). + + Examples + -------- + >>> _ = spark.sql("DROP TABLE IF EXISTS tbl_part_doc") + >>> _ = spark.sql( + ... "CREATE TABLE tbl_part_doc (id INT, p INT) USING parquet PARTITIONED BY (p)" + ... ) + >>> _ = spark.sql("INSERT INTO tbl_part_doc PARTITION (p = 1) SELECT 1") + >>> parts = [x.partition for x in spark.catalog.listPartitions("tbl_part_doc")] + >>> any("p=1" in s for s in parts) + True + >>> _ = spark.sql("DROP TABLE tbl_part_doc") + """ + iter = self._jcatalog.listPartitions(tableName).toLocalIterator() + out: List[CatalogTablePartition] = [] + while iter.hasNext(): + j = iter.next() + out.append(CatalogTablePartition(partition=j.partition())) + return out + + def listViews(self, dbName: Optional[str] = None, pattern: Optional[str] = None) -> List[Table]: + """Lists views in a namespace. + + With no arguments, lists views in the current namespace. With ``dbName`` only, lists + views in that namespace (may be catalog-qualified). With ``pattern``, filters view names + with a SQL ``LIKE`` string; if ``pattern`` is given without ``dbName``, the current + database is used as the namespace. + + .. versionadded:: 4.2.0 + + Parameters + ---------- + dbName : str, optional + Namespace to list views from. May be qualified with catalog name. + pattern : str, optional + SQL ``LIKE`` pattern for view names. + + Returns + ------- + list + A list of :class:`Table` (same row shape as :meth:`Catalog.listTables`). + + Notes + ----- + Raises :class:`AnalysisException` if ``dbName`` names a namespace that does not exist. + + Examples + -------- + >>> _ = spark.sql("DROP VIEW IF EXISTS view_list_doc") + >>> _ = spark.sql("CREATE VIEW view_list_doc AS SELECT 1 AS c") + >>> "view_list_doc" in [v.name for v in spark.catalog.listViews()] + True + >>> "view_list_doc" in [v.name for v in spark.catalog.listViews(pattern="view_list_*")] + True + >>> _ = spark.sql("DROP VIEW view_list_doc") + """ + if pattern is not None and dbName is None: + dbName = self.currentDatabase() + if dbName is None: + iter = self._jcatalog.listViews().toLocalIterator() + elif pattern is None: + iter = self._jcatalog.listViews(dbName).toLocalIterator() + else: + iter = self._jcatalog.listViews(dbName, pattern).toLocalIterator() + views = [] + while iter.hasNext(): + jtable = iter.next() + jnamespace = jtable.namespace() + if jnamespace is not None: + namespace = [jnamespace[i] for i in range(0, len(jnamespace))] + else: + namespace = None + views.append( + Table( + name=jtable.name(), + catalog=jtable.catalog(), + namespace=namespace, + description=jtable.description(), + tableType=jtable.tableType(), + isTemporary=jtable.isTemporary(), + ) + ) + return views + + def getTableProperties(self, tableName: str) -> Dict[str, str]: + """Returns all table properties as a dict (same as ``SHOW TBLPROPERTIES``). + + .. versionadded:: 4.2.0 + + Parameters + ---------- + tableName : str + Table or view name. May be qualified with catalog and database (namespace). + + Returns + ------- + dict + Map of property key to value. + + Examples + -------- + >>> _ = spark.sql("DROP TABLE IF EXISTS tbl_props_doc") + >>> _ = spark.sql( + ... "CREATE TABLE tbl_props_doc (id INT) USING parquet " + ... "TBLPROPERTIES ('doc_prop_key' = 'doc_prop_val')" + ... ) + >>> spark.catalog.getTableProperties("tbl_props_doc")["doc_prop_key"] + 'doc_prop_val' + >>> _ = spark.sql("DROP TABLE tbl_props_doc") + """ + jm = self._jcatalog.getTableProperties(tableName) + return {k: jm.get(k) for k in jm.keySet()} + + def getCreateTableString(self, tableName: str, asSerde: bool = False) -> str: + """Returns the ``SHOW CREATE TABLE`` DDL string for a relation. + + .. versionadded:: 4.2.0 + + Parameters + ---------- + tableName : str + Table or view name. May be qualified with catalog and database (namespace). + asSerde : bool, optional + If ``True``, request Hive serde DDL when applicable. + + Returns + ------- + str + DDL string from ``SHOW CREATE TABLE``. + + Examples + -------- + >>> _ = spark.sql("DROP TABLE IF EXISTS tbl_ddl_doc") + >>> _ = spark.sql("CREATE TABLE tbl_ddl_doc (id INT) USING parquet") + >>> "CREATE" in spark.catalog.getCreateTableString("tbl_ddl_doc").upper() + True + >>> _ = spark.sql("DROP TABLE tbl_ddl_doc") + """ + return self._jcatalog.getCreateTableString(tableName, asSerde) + + def truncateTable(self, tableName: str) -> None: + """Truncates a table (removes all data from the table; not supported for views). + + .. versionadded:: 4.2.0 + + Parameters + ---------- + tableName : str + Name of the table to truncate. May be qualified with catalog and database (namespace). + + Examples + -------- + >>> _ = spark.sql("DROP TABLE IF EXISTS tbl_tr_doc") + >>> _ = spark.sql("CREATE TABLE tbl_tr_doc (id INT) USING csv") + >>> _ = spark.sql("INSERT INTO tbl_tr_doc VALUES (1), (2)") + >>> spark.table("tbl_tr_doc").count() + 2 + >>> spark.catalog.truncateTable("tbl_tr_doc") + >>> spark.table("tbl_tr_doc").count() + 0 + >>> _ = spark.sql("DROP TABLE tbl_tr_doc") + """ + self._jcatalog.truncateTable(tableName) + + def analyzeTable(self, tableName: str, noScan: bool = False) -> None: + """Computes table statistics (same as SQL ``ANALYZE TABLE COMPUTE STATISTICS``). + + .. versionadded:: 4.2.0 + + Parameters + ---------- + tableName : str + Table or view name. May be qualified with catalog and database (namespace). + noScan : bool, optional + If ``True``, use ``NOSCAN`` mode (reuse existing column statistics where possible). + + Examples + -------- + >>> _ = spark.sql("DROP TABLE IF EXISTS tbl_an_doc") + >>> _ = spark.sql("CREATE TABLE tbl_an_doc (id INT) USING csv") + >>> _ = spark.sql("INSERT INTO tbl_an_doc VALUES (1)") + >>> spark.catalog.analyzeTable("tbl_an_doc") + >>> spark.catalog.analyzeTable("tbl_an_doc", noScan=True) + >>> _ = spark.sql("DROP TABLE tbl_an_doc") + """ + self._jcatalog.analyzeTable(tableName, noScan) + def currentDatabase(self) -> str: """ - Returns the current default database in this session. + Returns the current database (namespace) in this session. .. versionadded:: 2.0.0 Returns ------- str - The current default database name. + The current database (namespace) name. Examples -------- @@ -181,10 +543,15 @@ def currentDatabase(self) -> str: def setCurrentDatabase(self, dbName: str) -> None: """ - Sets the current default database in this session. + Sets the current database (namespace) in this session. .. versionadded:: 2.0.0 + Parameters + ---------- + dbName : str + Name of the database (namespace) to set. + Examples -------- >>> spark.catalog.setCurrentDatabase("default") @@ -193,16 +560,18 @@ def setCurrentDatabase(self, dbName: str) -> None: def listDatabases(self, pattern: Optional[str] = None) -> List[Database]: """ - Returns a list of databases available across all sessions. + Returns a list of databases (namespaces) available within the current catalog. + + With ``pattern``, returns only databases whose name matches that pattern. .. versionadded:: 2.0.0 Parameters ---------- pattern : str, optional - The pattern that the database name needs to match. + Pattern that database names must match. - .. versionadded: 3.5.0 + .. versionadded:: 3.5.0 Returns ------- @@ -310,33 +679,32 @@ def databaseExists(self, dbName: str) -> bool: def listTables( self, dbName: Optional[str] = None, pattern: Optional[str] = None ) -> List[Table]: - """Returns a list of tables/views in the specified database. + """Returns a list of tables/views in the current database (namespace), or in the database + given by ``dbName`` when provided (the name may be qualified with catalog). + + With ``pattern``, returns only tables and views whose name matches the pattern. This + includes all temporary views. .. versionadded:: 2.0.0 Parameters ---------- dbName : str, optional - name of the database to list the tables. + Database (namespace) to list tables from. If omitted, the current database is used. .. versionchanged:: 3.4.0 Allow ``dbName`` to be qualified with catalog name. pattern : str, optional - The pattern that the database name needs to match. + Pattern that table names must match. - .. versionadded: 3.5.0 + .. versionadded:: 3.5.0 Returns ------- list A list of :class:`Table`. - Notes - ----- - If no database is specified, the current database and catalog - are used. This API includes all temporary views. - Examples -------- >>> spark.range(1).createTempView("test_view") @@ -442,30 +810,29 @@ def listFunctions( self, dbName: Optional[str] = None, pattern: Optional[str] = None ) -> List[Function]: """ - Returns a list of functions registered in the specified database. + Returns a list of functions registered in the current database (namespace), or in the + database given by ``dbName`` when provided (the name may be qualified with catalog). + + With ``pattern``, returns only functions whose name matches the pattern. This includes all + built-in and temporary functions. .. versionadded:: 3.4.0 Parameters ---------- dbName : str, optional - name of the database to list the functions. - ``dbName`` can be qualified with catalog name. + Database (namespace) to list functions from. If omitted, the current database is used. + May be qualified with catalog name. pattern : str, optional - The pattern that the function name needs to match. + Pattern that function names must match. - .. versionadded: 3.5.0 + .. versionadded:: 3.5.0 Returns ------- list A list of :class:`Function`. - Notes - ----- - If no database is specified, the current database and catalog - are used. This API includes all temporary functions. - Examples -------- >>> spark.catalog.listFunctions() @@ -1166,18 +1533,17 @@ def refreshTable(self, tableName: str) -> None: self._jcatalog.refreshTable(tableName) def recoverPartitions(self, tableName: str) -> None: - """Recovers all the partitions of the given table and updates the catalog. + """Recovers all the partitions in the directory of a table and updates the catalog. + + Only works with a partitioned table, and not a view. .. versionadded:: 2.1.1 Parameters ---------- tableName : str - name of the table to get. - - Notes - ----- - Only works with a partitioned table, and not a view. + Table name; may be qualified with catalog and database (namespace). If no database + identifier is provided, the name refers to a table in the current database. Examples -------- @@ -1187,11 +1553,13 @@ def recoverPartitions(self, tableName: str) -> None: >>> import tempfile >>> with tempfile.TemporaryDirectory(prefix="recoverPartitions") as d: ... _ = spark.sql("DROP TABLE IF EXISTS tbl1") + ... p = d.replace("'", "''") ... spark.range(1).selectExpr( - ... "id as key", "id as value").write.partitionBy("key").mode("overwrite").save(d) + ... "id as key", "id as value").write.partitionBy( + ... "key").mode("overwrite").format("csv").save(d) ... _ = spark.sql( - ... "CREATE TABLE tbl1 (key LONG, value LONG)" - ... "USING parquet OPTIONS (path '{}') PARTITIONED BY (key)".format(d)) + ... "CREATE TABLE tbl1 (key LONG, value LONG) USING csv OPTIONS (header false, path '{}') " + ... "PARTITIONED BY (key)".format(p)) ... spark.table("tbl1").show() ... spark.catalog.recoverPartitions("tbl1") ... spark.table("tbl1").show() diff --git a/python/pyspark/sql/connect/catalog.py b/python/pyspark/sql/connect/catalog.py index 6bb418230e9e2..034b3fa1683c0 100644 --- a/python/pyspark/sql/connect/catalog.py +++ b/python/pyspark/sql/connect/catalog.py @@ -16,7 +16,7 @@ # from pyspark.errors import PySparkTypeError -from typing import Any, Callable, List, Optional, TYPE_CHECKING +from typing import Any, Callable, Dict, List, Optional, TYPE_CHECKING import warnings import pyarrow as pa @@ -25,8 +25,10 @@ from pyspark.sql.types import StructType from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.catalog import ( + CachedTable, Catalog as PySparkCatalog, CatalogMetadata, + CatalogTablePartition, Database, Table, Function, @@ -319,6 +321,90 @@ def refreshByPath(self, path: str) -> None: refreshByPath.__doc__ = PySparkCatalog.refreshByPath.__doc__ + def listCachedTables(self) -> List[CachedTable]: + table = self._execute_and_fetch(plan.ListCachedTables()) + return [ + CachedTable(table[0][i].as_py(), table[1][i].as_py()) for i in range(table.num_rows) + ] + + listCachedTables.__doc__ = PySparkCatalog.listCachedTables.__doc__ + + def dropTable(self, tableName: str, ifExists: bool = False, purge: bool = False) -> None: + self._execute_and_fetch( + plan.DropTable(table_name=tableName, if_exists=ifExists, purge=purge) + ) + + dropTable.__doc__ = PySparkCatalog.dropTable.__doc__ + + def dropView(self, viewName: str, ifExists: bool = False) -> None: + self._execute_and_fetch(plan.DropView(view_name=viewName, if_exists=ifExists)) + + dropView.__doc__ = PySparkCatalog.dropView.__doc__ + + def createDatabase( + self, dbName: str, ifNotExists: bool = False, properties: Optional[Dict[str, str]] = None + ) -> None: + self._execute_and_fetch( + plan.CreateDatabase(db_name=dbName, if_not_exists=ifNotExists, properties=properties) + ) + + createDatabase.__doc__ = PySparkCatalog.createDatabase.__doc__ + + def dropDatabase(self, dbName: str, ifExists: bool = False, cascade: bool = False) -> None: + self._execute_and_fetch( + plan.DropDatabase(db_name=dbName, if_exists=ifExists, cascade=cascade) + ) + + dropDatabase.__doc__ = PySparkCatalog.dropDatabase.__doc__ + + def listPartitions(self, tableName: str) -> List[CatalogTablePartition]: + table = self._execute_and_fetch(plan.ListPartitions(table_name=tableName)) + return [CatalogTablePartition(table[0][i].as_py()) for i in range(table.num_rows)] + + listPartitions.__doc__ = PySparkCatalog.listPartitions.__doc__ + + def listViews(self, dbName: Optional[str] = None, pattern: Optional[str] = None) -> List[Table]: + if pattern is not None and dbName is None: + dbName = self.currentDatabase() + table = self._execute_and_fetch(plan.ListViews(db_name=dbName, pattern=pattern)) + return [ + Table( + name=table[0][i].as_py(), + catalog=table[1][i].as_py(), + namespace=table[2][i].as_py(), + description=table[3][i].as_py(), + tableType=table[4][i].as_py(), + isTemporary=table[5][i].as_py(), + ) + for i in range(table.num_rows) + ] + + listViews.__doc__ = PySparkCatalog.listViews.__doc__ + + def getTableProperties(self, tableName: str) -> Dict[str, str]: + t = self._execute_and_fetch(plan.GetTableProperties(table_name=tableName)) + return {t[0][i].as_py(): t[1][i].as_py() for i in range(t.num_rows)} + + getTableProperties.__doc__ = PySparkCatalog.getTableProperties.__doc__ + + def getCreateTableString(self, tableName: str, asSerde: bool = False) -> str: + t = self._execute_and_fetch( + plan.GetCreateTableString(table_name=tableName, as_serde=asSerde) + ) + return t[0][0].as_py() if t.num_rows > 0 else "" + + getCreateTableString.__doc__ = PySparkCatalog.getCreateTableString.__doc__ + + def truncateTable(self, tableName: str) -> None: + self._execute_and_fetch(plan.TruncateTable(table_name=tableName)) + + truncateTable.__doc__ = PySparkCatalog.truncateTable.__doc__ + + def analyzeTable(self, tableName: str, noScan: bool = False) -> None: + self._execute_and_fetch(plan.AnalyzeTable(table_name=tableName, no_scan=noScan)) + + analyzeTable.__doc__ = PySparkCatalog.analyzeTable.__doc__ + def registerFunction( self, name: str, f: Callable[..., Any], returnType: Optional["DataTypeOrString"] = None ) -> "UserDefinedFunctionLike": diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 8b91517c85fc6..66e380c304d8a 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2482,6 +2482,167 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: return plan +class ListCachedTables(LogicalPlan): + def __init__(self) -> None: + super().__init__(None) + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.list_cached_tables.SetInParent() + return plan + + +class DropTable(LogicalPlan): + def __init__(self, table_name: str, if_exists: bool = False, purge: bool = False) -> None: + super().__init__(None) + self._table_name = table_name + self._if_exists = if_exists + self._purge = purge + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.drop_table.CopyFrom( + proto.DropTable( + table_name=self._table_name, + if_exists=self._if_exists, + purge=self._purge, + ) + ) + return plan + + +class DropView(LogicalPlan): + def __init__(self, view_name: str, if_exists: bool = False) -> None: + super().__init__(None) + self._view_name = view_name + self._if_exists = if_exists + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.drop_view.CopyFrom( + proto.DropView(view_name=self._view_name, if_exists=self._if_exists) + ) + return plan + + +class CreateDatabase(LogicalPlan): + def __init__( + self, + db_name: str, + if_not_exists: bool = False, + properties: Optional[Dict[str, str]] = None, + ) -> None: + super().__init__(None) + self._db_name = db_name + self._if_not_exists = if_not_exists + self._properties = properties or {} + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + cmd = proto.CreateDatabase( + db_name=self._db_name, + if_not_exists=self._if_not_exists, + ) + for k, v in self._properties.items(): + cmd.properties[k] = v + plan.catalog.create_database.CopyFrom(cmd) + return plan + + +class DropDatabase(LogicalPlan): + def __init__(self, db_name: str, if_exists: bool = False, cascade: bool = False) -> None: + super().__init__(None) + self._db_name = db_name + self._if_exists = if_exists + self._cascade = cascade + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.drop_database.CopyFrom( + proto.DropDatabase( + db_name=self._db_name, + if_exists=self._if_exists, + cascade=self._cascade, + ) + ) + return plan + + +class ListPartitions(LogicalPlan): + def __init__(self, table_name: str) -> None: + super().__init__(None) + self._table_name = table_name + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.list_partitions.table_name = self._table_name + return plan + + +class ListViews(LogicalPlan): + def __init__(self, db_name: Optional[str] = None, pattern: Optional[str] = None) -> None: + super().__init__(None) + self._db_name = db_name + self._pattern = pattern + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.list_views.SetInParent() + if self._db_name is not None: + plan.catalog.list_views.db_name = self._db_name + if self._pattern is not None: + plan.catalog.list_views.pattern = self._pattern + return plan + + +class GetTableProperties(LogicalPlan): + def __init__(self, table_name: str) -> None: + super().__init__(None) + self._table_name = table_name + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.get_table_properties.table_name = self._table_name + return plan + + +class GetCreateTableString(LogicalPlan): + def __init__(self, table_name: str, as_serde: bool = False) -> None: + super().__init__(None) + self._table_name = table_name + self._as_serde = as_serde + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.get_create_table_string.table_name = self._table_name + plan.catalog.get_create_table_string.as_serde = self._as_serde + return plan + + +class TruncateTable(LogicalPlan): + def __init__(self, table_name: str) -> None: + super().__init__(None) + self._table_name = table_name + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.truncate_table.table_name = self._table_name + return plan + + +class AnalyzeTable(LogicalPlan): + def __init__(self, table_name: str, no_scan: bool = False) -> None: + super().__init__(None) + self._table_name = table_name + self._no_scan = no_scan + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.catalog.analyze_table.table_name = self._table_name + plan.catalog.analyze_table.no_scan = self._no_scan + return plan + + class MapPartitions(LogicalPlan): """Logical plan object for a mapPartitions-equivalent API: mapInPandas, mapInArrow.""" diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.py b/python/pyspark/sql/connect/proto/catalog_pb2.py index c286def280ceb..5998afe4bee2b 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.py +++ b/python/pyspark/sql/connect/proto/catalog_pb2.py @@ -40,7 +40,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1bspark/connect/catalog.proto\x12\rspark.connect\x1a\x1aspark/connect/common.proto\x1a\x19spark/connect/types.proto"\xc6\x0e\n\x07\x43\x61talog\x12K\n\x10\x63urrent_database\x18\x01 \x01(\x0b\x32\x1e.spark.connect.CurrentDatabaseH\x00R\x0f\x63urrentDatabase\x12U\n\x14set_current_database\x18\x02 \x01(\x0b\x32!.spark.connect.SetCurrentDatabaseH\x00R\x12setCurrentDatabase\x12\x45\n\x0elist_databases\x18\x03 \x01(\x0b\x32\x1c.spark.connect.ListDatabasesH\x00R\rlistDatabases\x12<\n\x0blist_tables\x18\x04 \x01(\x0b\x32\x19.spark.connect.ListTablesH\x00R\nlistTables\x12\x45\n\x0elist_functions\x18\x05 \x01(\x0b\x32\x1c.spark.connect.ListFunctionsH\x00R\rlistFunctions\x12?\n\x0clist_columns\x18\x06 \x01(\x0b\x32\x1a.spark.connect.ListColumnsH\x00R\x0blistColumns\x12?\n\x0cget_database\x18\x07 \x01(\x0b\x32\x1a.spark.connect.GetDatabaseH\x00R\x0bgetDatabase\x12\x36\n\tget_table\x18\x08 \x01(\x0b\x32\x17.spark.connect.GetTableH\x00R\x08getTable\x12?\n\x0cget_function\x18\t \x01(\x0b\x32\x1a.spark.connect.GetFunctionH\x00R\x0bgetFunction\x12H\n\x0f\x64\x61tabase_exists\x18\n \x01(\x0b\x32\x1d.spark.connect.DatabaseExistsH\x00R\x0e\x64\x61tabaseExists\x12?\n\x0ctable_exists\x18\x0b \x01(\x0b\x32\x1a.spark.connect.TableExistsH\x00R\x0btableExists\x12H\n\x0f\x66unction_exists\x18\x0c \x01(\x0b\x32\x1d.spark.connect.FunctionExistsH\x00R\x0e\x66unctionExists\x12X\n\x15\x63reate_external_table\x18\r \x01(\x0b\x32".spark.connect.CreateExternalTableH\x00R\x13\x63reateExternalTable\x12?\n\x0c\x63reate_table\x18\x0e \x01(\x0b\x32\x1a.spark.connect.CreateTableH\x00R\x0b\x63reateTable\x12\x43\n\x0e\x64rop_temp_view\x18\x0f \x01(\x0b\x32\x1b.spark.connect.DropTempViewH\x00R\x0c\x64ropTempView\x12V\n\x15\x64rop_global_temp_view\x18\x10 \x01(\x0b\x32!.spark.connect.DropGlobalTempViewH\x00R\x12\x64ropGlobalTempView\x12Q\n\x12recover_partitions\x18\x11 \x01(\x0b\x32 .spark.connect.RecoverPartitionsH\x00R\x11recoverPartitions\x12\x36\n\tis_cached\x18\x12 \x01(\x0b\x32\x17.spark.connect.IsCachedH\x00R\x08isCached\x12<\n\x0b\x63\x61\x63he_table\x18\x13 \x01(\x0b\x32\x19.spark.connect.CacheTableH\x00R\ncacheTable\x12\x42\n\runcache_table\x18\x14 \x01(\x0b\x32\x1b.spark.connect.UncacheTableH\x00R\x0cuncacheTable\x12<\n\x0b\x63lear_cache\x18\x15 \x01(\x0b\x32\x19.spark.connect.ClearCacheH\x00R\nclearCache\x12\x42\n\rrefresh_table\x18\x16 \x01(\x0b\x32\x1b.spark.connect.RefreshTableH\x00R\x0crefreshTable\x12\x46\n\x0frefresh_by_path\x18\x17 \x01(\x0b\x32\x1c.spark.connect.RefreshByPathH\x00R\rrefreshByPath\x12H\n\x0f\x63urrent_catalog\x18\x18 \x01(\x0b\x32\x1d.spark.connect.CurrentCatalogH\x00R\x0e\x63urrentCatalog\x12R\n\x13set_current_catalog\x18\x19 \x01(\x0b\x32 .spark.connect.SetCurrentCatalogH\x00R\x11setCurrentCatalog\x12\x42\n\rlist_catalogs\x18\x1a \x01(\x0b\x32\x1b.spark.connect.ListCatalogsH\x00R\x0clistCatalogsB\n\n\x08\x63\x61t_type"\x11\n\x0f\x43urrentDatabase"-\n\x12SetCurrentDatabase\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name":\n\rListDatabases\x12\x1d\n\x07pattern\x18\x01 \x01(\tH\x00R\x07pattern\x88\x01\x01\x42\n\n\x08_pattern"a\n\nListTables\x12\x1c\n\x07\x64\x62_name\x18\x01 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x12\x1d\n\x07pattern\x18\x02 \x01(\tH\x01R\x07pattern\x88\x01\x01\x42\n\n\x08_db_nameB\n\n\x08_pattern"d\n\rListFunctions\x12\x1c\n\x07\x64\x62_name\x18\x01 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x12\x1d\n\x07pattern\x18\x02 \x01(\tH\x01R\x07pattern\x88\x01\x01\x42\n\n\x08_db_nameB\n\n\x08_pattern"V\n\x0bListColumns\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"&\n\x0bGetDatabase\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name"S\n\x08GetTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"\\\n\x0bGetFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name")\n\x0e\x44\x61tabaseExists\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name"V\n\x0bTableExists\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"_\n\x0e\x46unctionExists\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"\xc6\x02\n\x13\x43reateExternalTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x17\n\x04path\x18\x02 \x01(\tH\x00R\x04path\x88\x01\x01\x12\x1b\n\x06source\x18\x03 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x34\n\x06schema\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x02R\x06schema\x88\x01\x01\x12I\n\x07options\x18\x05 \x03(\x0b\x32/.spark.connect.CreateExternalTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x07\n\x05_pathB\t\n\x07_sourceB\t\n\x07_schema"\xed\x02\n\x0b\x43reateTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x17\n\x04path\x18\x02 \x01(\tH\x00R\x04path\x88\x01\x01\x12\x1b\n\x06source\x18\x03 \x01(\tH\x01R\x06source\x88\x01\x01\x12%\n\x0b\x64\x65scription\x18\x04 \x01(\tH\x02R\x0b\x64\x65scription\x88\x01\x01\x12\x34\n\x06schema\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x06schema\x88\x01\x01\x12\x41\n\x07options\x18\x06 \x03(\x0b\x32\'.spark.connect.CreateTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x07\n\x05_pathB\t\n\x07_sourceB\x0e\n\x0c_descriptionB\t\n\x07_schema"+\n\x0c\x44ropTempView\x12\x1b\n\tview_name\x18\x01 \x01(\tR\x08viewName"1\n\x12\x44ropGlobalTempView\x12\x1b\n\tview_name\x18\x01 \x01(\tR\x08viewName"2\n\x11RecoverPartitions\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName")\n\x08IsCached\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"\x84\x01\n\nCacheTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level"-\n\x0cUncacheTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"\x0c\n\nClearCache"-\n\x0cRefreshTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"#\n\rRefreshByPath\x12\x12\n\x04path\x18\x01 \x01(\tR\x04path"\x10\n\x0e\x43urrentCatalog"6\n\x11SetCurrentCatalog\x12!\n\x0c\x63\x61talog_name\x18\x01 \x01(\tR\x0b\x63\x61talogName"9\n\x0cListCatalogs\x12\x1d\n\x07pattern\x18\x01 \x01(\tH\x00R\x07pattern\x88\x01\x01\x42\n\n\x08_patternB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1bspark/connect/catalog.proto\x12\rspark.connect\x1a\x1aspark/connect/common.proto\x1a\x19spark/connect/types.proto"\xdd\x14\n\x07\x43\x61talog\x12K\n\x10\x63urrent_database\x18\x01 \x01(\x0b\x32\x1e.spark.connect.CurrentDatabaseH\x00R\x0f\x63urrentDatabase\x12U\n\x14set_current_database\x18\x02 \x01(\x0b\x32!.spark.connect.SetCurrentDatabaseH\x00R\x12setCurrentDatabase\x12\x45\n\x0elist_databases\x18\x03 \x01(\x0b\x32\x1c.spark.connect.ListDatabasesH\x00R\rlistDatabases\x12<\n\x0blist_tables\x18\x04 \x01(\x0b\x32\x19.spark.connect.ListTablesH\x00R\nlistTables\x12\x45\n\x0elist_functions\x18\x05 \x01(\x0b\x32\x1c.spark.connect.ListFunctionsH\x00R\rlistFunctions\x12?\n\x0clist_columns\x18\x06 \x01(\x0b\x32\x1a.spark.connect.ListColumnsH\x00R\x0blistColumns\x12?\n\x0cget_database\x18\x07 \x01(\x0b\x32\x1a.spark.connect.GetDatabaseH\x00R\x0bgetDatabase\x12\x36\n\tget_table\x18\x08 \x01(\x0b\x32\x17.spark.connect.GetTableH\x00R\x08getTable\x12?\n\x0cget_function\x18\t \x01(\x0b\x32\x1a.spark.connect.GetFunctionH\x00R\x0bgetFunction\x12H\n\x0f\x64\x61tabase_exists\x18\n \x01(\x0b\x32\x1d.spark.connect.DatabaseExistsH\x00R\x0e\x64\x61tabaseExists\x12?\n\x0ctable_exists\x18\x0b \x01(\x0b\x32\x1a.spark.connect.TableExistsH\x00R\x0btableExists\x12H\n\x0f\x66unction_exists\x18\x0c \x01(\x0b\x32\x1d.spark.connect.FunctionExistsH\x00R\x0e\x66unctionExists\x12X\n\x15\x63reate_external_table\x18\r \x01(\x0b\x32".spark.connect.CreateExternalTableH\x00R\x13\x63reateExternalTable\x12?\n\x0c\x63reate_table\x18\x0e \x01(\x0b\x32\x1a.spark.connect.CreateTableH\x00R\x0b\x63reateTable\x12\x43\n\x0e\x64rop_temp_view\x18\x0f \x01(\x0b\x32\x1b.spark.connect.DropTempViewH\x00R\x0c\x64ropTempView\x12V\n\x15\x64rop_global_temp_view\x18\x10 \x01(\x0b\x32!.spark.connect.DropGlobalTempViewH\x00R\x12\x64ropGlobalTempView\x12Q\n\x12recover_partitions\x18\x11 \x01(\x0b\x32 .spark.connect.RecoverPartitionsH\x00R\x11recoverPartitions\x12\x36\n\tis_cached\x18\x12 \x01(\x0b\x32\x17.spark.connect.IsCachedH\x00R\x08isCached\x12<\n\x0b\x63\x61\x63he_table\x18\x13 \x01(\x0b\x32\x19.spark.connect.CacheTableH\x00R\ncacheTable\x12\x42\n\runcache_table\x18\x14 \x01(\x0b\x32\x1b.spark.connect.UncacheTableH\x00R\x0cuncacheTable\x12<\n\x0b\x63lear_cache\x18\x15 \x01(\x0b\x32\x19.spark.connect.ClearCacheH\x00R\nclearCache\x12\x42\n\rrefresh_table\x18\x16 \x01(\x0b\x32\x1b.spark.connect.RefreshTableH\x00R\x0crefreshTable\x12\x46\n\x0frefresh_by_path\x18\x17 \x01(\x0b\x32\x1c.spark.connect.RefreshByPathH\x00R\rrefreshByPath\x12H\n\x0f\x63urrent_catalog\x18\x18 \x01(\x0b\x32\x1d.spark.connect.CurrentCatalogH\x00R\x0e\x63urrentCatalog\x12R\n\x13set_current_catalog\x18\x19 \x01(\x0b\x32 .spark.connect.SetCurrentCatalogH\x00R\x11setCurrentCatalog\x12\x42\n\rlist_catalogs\x18\x1a \x01(\x0b\x32\x1b.spark.connect.ListCatalogsH\x00R\x0clistCatalogs\x12O\n\x12list_cached_tables\x18\x1b \x01(\x0b\x32\x1f.spark.connect.ListCachedTablesH\x00R\x10listCachedTables\x12\x39\n\ndrop_table\x18\x1c \x01(\x0b\x32\x18.spark.connect.DropTableH\x00R\tdropTable\x12\x36\n\tdrop_view\x18\x1d \x01(\x0b\x32\x17.spark.connect.DropViewH\x00R\x08\x64ropView\x12H\n\x0f\x63reate_database\x18\x1e \x01(\x0b\x32\x1d.spark.connect.CreateDatabaseH\x00R\x0e\x63reateDatabase\x12\x42\n\rdrop_database\x18\x1f \x01(\x0b\x32\x1b.spark.connect.DropDatabaseH\x00R\x0c\x64ropDatabase\x12H\n\x0flist_partitions\x18 \x01(\x0b\x32\x1d.spark.connect.ListPartitionsH\x00R\x0elistPartitions\x12\x39\n\nlist_views\x18! \x01(\x0b\x32\x18.spark.connect.ListViewsH\x00R\tlistViews\x12U\n\x14get_table_properties\x18" \x01(\x0b\x32!.spark.connect.GetTablePropertiesH\x00R\x12getTableProperties\x12\\\n\x17get_create_table_string\x18# \x01(\x0b\x32#.spark.connect.GetCreateTableStringH\x00R\x14getCreateTableString\x12\x45\n\x0etruncate_table\x18$ \x01(\x0b\x32\x1c.spark.connect.TruncateTableH\x00R\rtruncateTable\x12\x42\n\ranalyze_table\x18% \x01(\x0b\x32\x1b.spark.connect.AnalyzeTableH\x00R\x0c\x61nalyzeTableB\n\n\x08\x63\x61t_type"\x11\n\x0f\x43urrentDatabase"-\n\x12SetCurrentDatabase\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name":\n\rListDatabases\x12\x1d\n\x07pattern\x18\x01 \x01(\tH\x00R\x07pattern\x88\x01\x01\x42\n\n\x08_pattern"a\n\nListTables\x12\x1c\n\x07\x64\x62_name\x18\x01 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x12\x1d\n\x07pattern\x18\x02 \x01(\tH\x01R\x07pattern\x88\x01\x01\x42\n\n\x08_db_nameB\n\n\x08_pattern"d\n\rListFunctions\x12\x1c\n\x07\x64\x62_name\x18\x01 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x12\x1d\n\x07pattern\x18\x02 \x01(\tH\x01R\x07pattern\x88\x01\x01\x42\n\n\x08_db_nameB\n\n\x08_pattern"V\n\x0bListColumns\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"&\n\x0bGetDatabase\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name"S\n\x08GetTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"\\\n\x0bGetFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name")\n\x0e\x44\x61tabaseExists\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name"V\n\x0bTableExists\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"_\n\x0e\x46unctionExists\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x1c\n\x07\x64\x62_name\x18\x02 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x42\n\n\x08_db_name"\xc6\x02\n\x13\x43reateExternalTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x17\n\x04path\x18\x02 \x01(\tH\x00R\x04path\x88\x01\x01\x12\x1b\n\x06source\x18\x03 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x34\n\x06schema\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x02R\x06schema\x88\x01\x01\x12I\n\x07options\x18\x05 \x03(\x0b\x32/.spark.connect.CreateExternalTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x07\n\x05_pathB\t\n\x07_sourceB\t\n\x07_schema"\xed\x02\n\x0b\x43reateTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x17\n\x04path\x18\x02 \x01(\tH\x00R\x04path\x88\x01\x01\x12\x1b\n\x06source\x18\x03 \x01(\tH\x01R\x06source\x88\x01\x01\x12%\n\x0b\x64\x65scription\x18\x04 \x01(\tH\x02R\x0b\x64\x65scription\x88\x01\x01\x12\x34\n\x06schema\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x06schema\x88\x01\x01\x12\x41\n\x07options\x18\x06 \x03(\x0b\x32\'.spark.connect.CreateTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x07\n\x05_pathB\t\n\x07_sourceB\x0e\n\x0c_descriptionB\t\n\x07_schema"+\n\x0c\x44ropTempView\x12\x1b\n\tview_name\x18\x01 \x01(\tR\x08viewName"1\n\x12\x44ropGlobalTempView\x12\x1b\n\tview_name\x18\x01 \x01(\tR\x08viewName"2\n\x11RecoverPartitions\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName")\n\x08IsCached\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"\x84\x01\n\nCacheTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level"-\n\x0cUncacheTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"\x0c\n\nClearCache"-\n\x0cRefreshTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"#\n\rRefreshByPath\x12\x12\n\x04path\x18\x01 \x01(\tR\x04path"\x10\n\x0e\x43urrentCatalog"6\n\x11SetCurrentCatalog\x12!\n\x0c\x63\x61talog_name\x18\x01 \x01(\tR\x0b\x63\x61talogName"9\n\x0cListCatalogs\x12\x1d\n\x07pattern\x18\x01 \x01(\tH\x00R\x07pattern\x88\x01\x01\x42\n\n\x08_pattern"\x12\n\x10ListCachedTables"]\n\tDropTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x1b\n\tif_exists\x18\x02 \x01(\x08R\x08ifExists\x12\x14\n\x05purge\x18\x03 \x01(\x08R\x05purge"D\n\x08\x44ropView\x12\x1b\n\tview_name\x18\x01 \x01(\tR\x08viewName\x12\x1b\n\tif_exists\x18\x02 \x01(\x08R\x08ifExists"\xdb\x01\n\x0e\x43reateDatabase\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name\x12"\n\rif_not_exists\x18\x02 \x01(\x08R\x0bifNotExists\x12M\n\nproperties\x18\x03 \x03(\x0b\x32-.spark.connect.CreateDatabase.PropertiesEntryR\nproperties\x1a=\n\x0fPropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"^\n\x0c\x44ropDatabase\x12\x17\n\x07\x64\x62_name\x18\x01 \x01(\tR\x06\x64\x62Name\x12\x1b\n\tif_exists\x18\x02 \x01(\x08R\x08ifExists\x12\x18\n\x07\x63\x61scade\x18\x03 \x01(\x08R\x07\x63\x61scade"/\n\x0eListPartitions\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"`\n\tListViews\x12\x1c\n\x07\x64\x62_name\x18\x01 \x01(\tH\x00R\x06\x64\x62Name\x88\x01\x01\x12\x1d\n\x07pattern\x18\x02 \x01(\tH\x01R\x07pattern\x88\x01\x01\x42\n\n\x08_db_nameB\n\n\x08_pattern"3\n\x12GetTableProperties\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"P\n\x14GetCreateTableString\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x19\n\x08\x61s_serde\x18\x02 \x01(\x08R\x07\x61sSerde".\n\rTruncateTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName"F\n\x0c\x41nalyzeTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12\x17\n\x07no_scan\x18\x02 \x01(\x08R\x06noScanB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -57,62 +57,88 @@ _globals["_CREATEEXTERNALTABLE_OPTIONSENTRY"]._serialized_options = b"8\001" _globals["_CREATETABLE_OPTIONSENTRY"]._loaded_options = None _globals["_CREATETABLE_OPTIONSENTRY"]._serialized_options = b"8\001" + _globals["_CREATEDATABASE_PROPERTIESENTRY"]._loaded_options = None + _globals["_CREATEDATABASE_PROPERTIESENTRY"]._serialized_options = b"8\001" _globals["_CATALOG"]._serialized_start = 102 - _globals["_CATALOG"]._serialized_end = 1964 - _globals["_CURRENTDATABASE"]._serialized_start = 1966 - _globals["_CURRENTDATABASE"]._serialized_end = 1983 - _globals["_SETCURRENTDATABASE"]._serialized_start = 1985 - _globals["_SETCURRENTDATABASE"]._serialized_end = 2030 - _globals["_LISTDATABASES"]._serialized_start = 2032 - _globals["_LISTDATABASES"]._serialized_end = 2090 - _globals["_LISTTABLES"]._serialized_start = 2092 - _globals["_LISTTABLES"]._serialized_end = 2189 - _globals["_LISTFUNCTIONS"]._serialized_start = 2191 - _globals["_LISTFUNCTIONS"]._serialized_end = 2291 - _globals["_LISTCOLUMNS"]._serialized_start = 2293 - _globals["_LISTCOLUMNS"]._serialized_end = 2379 - _globals["_GETDATABASE"]._serialized_start = 2381 - _globals["_GETDATABASE"]._serialized_end = 2419 - _globals["_GETTABLE"]._serialized_start = 2421 - _globals["_GETTABLE"]._serialized_end = 2504 - _globals["_GETFUNCTION"]._serialized_start = 2506 - _globals["_GETFUNCTION"]._serialized_end = 2598 - _globals["_DATABASEEXISTS"]._serialized_start = 2600 - _globals["_DATABASEEXISTS"]._serialized_end = 2641 - _globals["_TABLEEXISTS"]._serialized_start = 2643 - _globals["_TABLEEXISTS"]._serialized_end = 2729 - _globals["_FUNCTIONEXISTS"]._serialized_start = 2731 - _globals["_FUNCTIONEXISTS"]._serialized_end = 2826 - _globals["_CREATEEXTERNALTABLE"]._serialized_start = 2829 - _globals["_CREATEEXTERNALTABLE"]._serialized_end = 3155 - _globals["_CREATEEXTERNALTABLE_OPTIONSENTRY"]._serialized_start = 3066 - _globals["_CREATEEXTERNALTABLE_OPTIONSENTRY"]._serialized_end = 3124 - _globals["_CREATETABLE"]._serialized_start = 3158 - _globals["_CREATETABLE"]._serialized_end = 3523 - _globals["_CREATETABLE_OPTIONSENTRY"]._serialized_start = 3066 - _globals["_CREATETABLE_OPTIONSENTRY"]._serialized_end = 3124 - _globals["_DROPTEMPVIEW"]._serialized_start = 3525 - _globals["_DROPTEMPVIEW"]._serialized_end = 3568 - _globals["_DROPGLOBALTEMPVIEW"]._serialized_start = 3570 - _globals["_DROPGLOBALTEMPVIEW"]._serialized_end = 3619 - _globals["_RECOVERPARTITIONS"]._serialized_start = 3621 - _globals["_RECOVERPARTITIONS"]._serialized_end = 3671 - _globals["_ISCACHED"]._serialized_start = 3673 - _globals["_ISCACHED"]._serialized_end = 3714 - _globals["_CACHETABLE"]._serialized_start = 3717 - _globals["_CACHETABLE"]._serialized_end = 3849 - _globals["_UNCACHETABLE"]._serialized_start = 3851 - _globals["_UNCACHETABLE"]._serialized_end = 3896 - _globals["_CLEARCACHE"]._serialized_start = 3898 - _globals["_CLEARCACHE"]._serialized_end = 3910 - _globals["_REFRESHTABLE"]._serialized_start = 3912 - _globals["_REFRESHTABLE"]._serialized_end = 3957 - _globals["_REFRESHBYPATH"]._serialized_start = 3959 - _globals["_REFRESHBYPATH"]._serialized_end = 3994 - _globals["_CURRENTCATALOG"]._serialized_start = 3996 - _globals["_CURRENTCATALOG"]._serialized_end = 4012 - _globals["_SETCURRENTCATALOG"]._serialized_start = 4014 - _globals["_SETCURRENTCATALOG"]._serialized_end = 4068 - _globals["_LISTCATALOGS"]._serialized_start = 4070 - _globals["_LISTCATALOGS"]._serialized_end = 4127 + _globals["_CATALOG"]._serialized_end = 2755 + _globals["_CURRENTDATABASE"]._serialized_start = 2757 + _globals["_CURRENTDATABASE"]._serialized_end = 2774 + _globals["_SETCURRENTDATABASE"]._serialized_start = 2776 + _globals["_SETCURRENTDATABASE"]._serialized_end = 2821 + _globals["_LISTDATABASES"]._serialized_start = 2823 + _globals["_LISTDATABASES"]._serialized_end = 2881 + _globals["_LISTTABLES"]._serialized_start = 2883 + _globals["_LISTTABLES"]._serialized_end = 2980 + _globals["_LISTFUNCTIONS"]._serialized_start = 2982 + _globals["_LISTFUNCTIONS"]._serialized_end = 3082 + _globals["_LISTCOLUMNS"]._serialized_start = 3084 + _globals["_LISTCOLUMNS"]._serialized_end = 3170 + _globals["_GETDATABASE"]._serialized_start = 3172 + _globals["_GETDATABASE"]._serialized_end = 3210 + _globals["_GETTABLE"]._serialized_start = 3212 + _globals["_GETTABLE"]._serialized_end = 3295 + _globals["_GETFUNCTION"]._serialized_start = 3297 + _globals["_GETFUNCTION"]._serialized_end = 3389 + _globals["_DATABASEEXISTS"]._serialized_start = 3391 + _globals["_DATABASEEXISTS"]._serialized_end = 3432 + _globals["_TABLEEXISTS"]._serialized_start = 3434 + _globals["_TABLEEXISTS"]._serialized_end = 3520 + _globals["_FUNCTIONEXISTS"]._serialized_start = 3522 + _globals["_FUNCTIONEXISTS"]._serialized_end = 3617 + _globals["_CREATEEXTERNALTABLE"]._serialized_start = 3620 + _globals["_CREATEEXTERNALTABLE"]._serialized_end = 3946 + _globals["_CREATEEXTERNALTABLE_OPTIONSENTRY"]._serialized_start = 3857 + _globals["_CREATEEXTERNALTABLE_OPTIONSENTRY"]._serialized_end = 3915 + _globals["_CREATETABLE"]._serialized_start = 3949 + _globals["_CREATETABLE"]._serialized_end = 4314 + _globals["_CREATETABLE_OPTIONSENTRY"]._serialized_start = 3857 + _globals["_CREATETABLE_OPTIONSENTRY"]._serialized_end = 3915 + _globals["_DROPTEMPVIEW"]._serialized_start = 4316 + _globals["_DROPTEMPVIEW"]._serialized_end = 4359 + _globals["_DROPGLOBALTEMPVIEW"]._serialized_start = 4361 + _globals["_DROPGLOBALTEMPVIEW"]._serialized_end = 4410 + _globals["_RECOVERPARTITIONS"]._serialized_start = 4412 + _globals["_RECOVERPARTITIONS"]._serialized_end = 4462 + _globals["_ISCACHED"]._serialized_start = 4464 + _globals["_ISCACHED"]._serialized_end = 4505 + _globals["_CACHETABLE"]._serialized_start = 4508 + _globals["_CACHETABLE"]._serialized_end = 4640 + _globals["_UNCACHETABLE"]._serialized_start = 4642 + _globals["_UNCACHETABLE"]._serialized_end = 4687 + _globals["_CLEARCACHE"]._serialized_start = 4689 + _globals["_CLEARCACHE"]._serialized_end = 4701 + _globals["_REFRESHTABLE"]._serialized_start = 4703 + _globals["_REFRESHTABLE"]._serialized_end = 4748 + _globals["_REFRESHBYPATH"]._serialized_start = 4750 + _globals["_REFRESHBYPATH"]._serialized_end = 4785 + _globals["_CURRENTCATALOG"]._serialized_start = 4787 + _globals["_CURRENTCATALOG"]._serialized_end = 4803 + _globals["_SETCURRENTCATALOG"]._serialized_start = 4805 + _globals["_SETCURRENTCATALOG"]._serialized_end = 4859 + _globals["_LISTCATALOGS"]._serialized_start = 4861 + _globals["_LISTCATALOGS"]._serialized_end = 4918 + _globals["_LISTCACHEDTABLES"]._serialized_start = 4920 + _globals["_LISTCACHEDTABLES"]._serialized_end = 4938 + _globals["_DROPTABLE"]._serialized_start = 4940 + _globals["_DROPTABLE"]._serialized_end = 5033 + _globals["_DROPVIEW"]._serialized_start = 5035 + _globals["_DROPVIEW"]._serialized_end = 5103 + _globals["_CREATEDATABASE"]._serialized_start = 5106 + _globals["_CREATEDATABASE"]._serialized_end = 5325 + _globals["_CREATEDATABASE_PROPERTIESENTRY"]._serialized_start = 5264 + _globals["_CREATEDATABASE_PROPERTIESENTRY"]._serialized_end = 5325 + _globals["_DROPDATABASE"]._serialized_start = 5327 + _globals["_DROPDATABASE"]._serialized_end = 5421 + _globals["_LISTPARTITIONS"]._serialized_start = 5423 + _globals["_LISTPARTITIONS"]._serialized_end = 5470 + _globals["_LISTVIEWS"]._serialized_start = 5472 + _globals["_LISTVIEWS"]._serialized_end = 5568 + _globals["_GETTABLEPROPERTIES"]._serialized_start = 5570 + _globals["_GETTABLEPROPERTIES"]._serialized_end = 5621 + _globals["_GETCREATETABLESTRING"]._serialized_start = 5623 + _globals["_GETCREATETABLESTRING"]._serialized_end = 5703 + _globals["_TRUNCATETABLE"]._serialized_start = 5705 + _globals["_TRUNCATETABLE"]._serialized_end = 5751 + _globals["_ANALYZETABLE"]._serialized_start = 5753 + _globals["_ANALYZETABLE"]._serialized_end = 5823 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.pyi b/python/pyspark/sql/connect/proto/catalog_pb2.pyi index 90864adbd5ca6..86eefa6a3724a 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.pyi +++ b/python/pyspark/sql/connect/proto/catalog_pb2.pyi @@ -82,6 +82,17 @@ class Catalog(google.protobuf.message.Message): CURRENT_CATALOG_FIELD_NUMBER: builtins.int SET_CURRENT_CATALOG_FIELD_NUMBER: builtins.int LIST_CATALOGS_FIELD_NUMBER: builtins.int + LIST_CACHED_TABLES_FIELD_NUMBER: builtins.int + DROP_TABLE_FIELD_NUMBER: builtins.int + DROP_VIEW_FIELD_NUMBER: builtins.int + CREATE_DATABASE_FIELD_NUMBER: builtins.int + DROP_DATABASE_FIELD_NUMBER: builtins.int + LIST_PARTITIONS_FIELD_NUMBER: builtins.int + LIST_VIEWS_FIELD_NUMBER: builtins.int + GET_TABLE_PROPERTIES_FIELD_NUMBER: builtins.int + GET_CREATE_TABLE_STRING_FIELD_NUMBER: builtins.int + TRUNCATE_TABLE_FIELD_NUMBER: builtins.int + ANALYZE_TABLE_FIELD_NUMBER: builtins.int @property def current_database(self) -> global___CurrentDatabase: ... @property @@ -134,6 +145,28 @@ class Catalog(google.protobuf.message.Message): def set_current_catalog(self) -> global___SetCurrentCatalog: ... @property def list_catalogs(self) -> global___ListCatalogs: ... + @property + def list_cached_tables(self) -> global___ListCachedTables: ... + @property + def drop_table(self) -> global___DropTable: ... + @property + def drop_view(self) -> global___DropView: ... + @property + def create_database(self) -> global___CreateDatabase: ... + @property + def drop_database(self) -> global___DropDatabase: ... + @property + def list_partitions(self) -> global___ListPartitions: ... + @property + def list_views(self) -> global___ListViews: ... + @property + def get_table_properties(self) -> global___GetTableProperties: ... + @property + def get_create_table_string(self) -> global___GetCreateTableString: ... + @property + def truncate_table(self) -> global___TruncateTable: ... + @property + def analyze_table(self) -> global___AnalyzeTable: ... def __init__( self, *, @@ -163,16 +196,31 @@ class Catalog(google.protobuf.message.Message): current_catalog: global___CurrentCatalog | None = ..., set_current_catalog: global___SetCurrentCatalog | None = ..., list_catalogs: global___ListCatalogs | None = ..., + list_cached_tables: global___ListCachedTables | None = ..., + drop_table: global___DropTable | None = ..., + drop_view: global___DropView | None = ..., + create_database: global___CreateDatabase | None = ..., + drop_database: global___DropDatabase | None = ..., + list_partitions: global___ListPartitions | None = ..., + list_views: global___ListViews | None = ..., + get_table_properties: global___GetTableProperties | None = ..., + get_create_table_string: global___GetCreateTableString | None = ..., + truncate_table: global___TruncateTable | None = ..., + analyze_table: global___AnalyzeTable | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ + "analyze_table", + b"analyze_table", "cache_table", b"cache_table", "cat_type", b"cat_type", "clear_cache", b"clear_cache", + "create_database", + b"create_database", "create_external_table", b"create_external_table", "create_table", @@ -183,20 +231,32 @@ class Catalog(google.protobuf.message.Message): b"current_database", "database_exists", b"database_exists", + "drop_database", + b"drop_database", "drop_global_temp_view", b"drop_global_temp_view", + "drop_table", + b"drop_table", "drop_temp_view", b"drop_temp_view", + "drop_view", + b"drop_view", "function_exists", b"function_exists", + "get_create_table_string", + b"get_create_table_string", "get_database", b"get_database", "get_function", b"get_function", "get_table", b"get_table", + "get_table_properties", + b"get_table_properties", "is_cached", b"is_cached", + "list_cached_tables", + b"list_cached_tables", "list_catalogs", b"list_catalogs", "list_columns", @@ -205,8 +265,12 @@ class Catalog(google.protobuf.message.Message): b"list_databases", "list_functions", b"list_functions", + "list_partitions", + b"list_partitions", "list_tables", b"list_tables", + "list_views", + b"list_views", "recover_partitions", b"recover_partitions", "refresh_by_path", @@ -219,6 +283,8 @@ class Catalog(google.protobuf.message.Message): b"set_current_database", "table_exists", b"table_exists", + "truncate_table", + b"truncate_table", "uncache_table", b"uncache_table", ], @@ -226,12 +292,16 @@ class Catalog(google.protobuf.message.Message): def ClearField( self, field_name: typing_extensions.Literal[ + "analyze_table", + b"analyze_table", "cache_table", b"cache_table", "cat_type", b"cat_type", "clear_cache", b"clear_cache", + "create_database", + b"create_database", "create_external_table", b"create_external_table", "create_table", @@ -242,20 +312,32 @@ class Catalog(google.protobuf.message.Message): b"current_database", "database_exists", b"database_exists", + "drop_database", + b"drop_database", "drop_global_temp_view", b"drop_global_temp_view", + "drop_table", + b"drop_table", "drop_temp_view", b"drop_temp_view", + "drop_view", + b"drop_view", "function_exists", b"function_exists", + "get_create_table_string", + b"get_create_table_string", "get_database", b"get_database", "get_function", b"get_function", "get_table", b"get_table", + "get_table_properties", + b"get_table_properties", "is_cached", b"is_cached", + "list_cached_tables", + b"list_cached_tables", "list_catalogs", b"list_catalogs", "list_columns", @@ -264,8 +346,12 @@ class Catalog(google.protobuf.message.Message): b"list_databases", "list_functions", b"list_functions", + "list_partitions", + b"list_partitions", "list_tables", b"list_tables", + "list_views", + b"list_views", "recover_partitions", b"recover_partitions", "refresh_by_path", @@ -278,6 +364,8 @@ class Catalog(google.protobuf.message.Message): b"set_current_database", "table_exists", b"table_exists", + "truncate_table", + b"truncate_table", "uncache_table", b"uncache_table", ], @@ -312,6 +400,17 @@ class Catalog(google.protobuf.message.Message): "current_catalog", "set_current_catalog", "list_catalogs", + "list_cached_tables", + "drop_table", + "drop_view", + "create_database", + "drop_database", + "list_partitions", + "list_views", + "get_table_properties", + "get_create_table_string", + "truncate_table", + "analyze_table", ] | None ): ... @@ -1132,3 +1231,298 @@ class ListCatalogs(google.protobuf.message.Message): ) -> typing_extensions.Literal["pattern"] | None: ... global___ListCatalogs = ListCatalogs + +class ListCachedTables(google.protobuf.message.Message): + """See `spark.catalog.listCachedTables`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + +global___ListCachedTables = ListCachedTables + +class DropTable(google.protobuf.message.Message): + """See `spark.catalog.dropTable`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TABLE_NAME_FIELD_NUMBER: builtins.int + IF_EXISTS_FIELD_NUMBER: builtins.int + PURGE_FIELD_NUMBER: builtins.int + table_name: builtins.str + """(Required)""" + if_exists: builtins.bool + purge: builtins.bool + def __init__( + self, + *, + table_name: builtins.str = ..., + if_exists: builtins.bool = ..., + purge: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "if_exists", b"if_exists", "purge", b"purge", "table_name", b"table_name" + ], + ) -> None: ... + +global___DropTable = DropTable + +class DropView(google.protobuf.message.Message): + """See `spark.catalog.dropView`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VIEW_NAME_FIELD_NUMBER: builtins.int + IF_EXISTS_FIELD_NUMBER: builtins.int + view_name: builtins.str + """(Required)""" + if_exists: builtins.bool + def __init__( + self, + *, + view_name: builtins.str = ..., + if_exists: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal["if_exists", b"if_exists", "view_name", b"view_name"], + ) -> None: ... + +global___DropView = DropView + +class CreateDatabase(google.protobuf.message.Message): + """See `spark.catalog.createDatabase`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class PropertiesEntry(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + value: builtins.str + def __init__( + self, + *, + key: builtins.str = ..., + value: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + + DB_NAME_FIELD_NUMBER: builtins.int + IF_NOT_EXISTS_FIELD_NUMBER: builtins.int + PROPERTIES_FIELD_NUMBER: builtins.int + db_name: builtins.str + """(Required)""" + if_not_exists: builtins.bool + @property + def properties( + self, + ) -> google.protobuf.internal.containers.ScalarMap[builtins.str, builtins.str]: ... + def __init__( + self, + *, + db_name: builtins.str = ..., + if_not_exists: builtins.bool = ..., + properties: collections.abc.Mapping[builtins.str, builtins.str] | None = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "db_name", b"db_name", "if_not_exists", b"if_not_exists", "properties", b"properties" + ], + ) -> None: ... + +global___CreateDatabase = CreateDatabase + +class DropDatabase(google.protobuf.message.Message): + """See `spark.catalog.dropDatabase`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + DB_NAME_FIELD_NUMBER: builtins.int + IF_EXISTS_FIELD_NUMBER: builtins.int + CASCADE_FIELD_NUMBER: builtins.int + db_name: builtins.str + """(Required)""" + if_exists: builtins.bool + cascade: builtins.bool + def __init__( + self, + *, + db_name: builtins.str = ..., + if_exists: builtins.bool = ..., + cascade: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "cascade", b"cascade", "db_name", b"db_name", "if_exists", b"if_exists" + ], + ) -> None: ... + +global___DropDatabase = DropDatabase + +class ListPartitions(google.protobuf.message.Message): + """See `spark.catalog.listPartitions`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TABLE_NAME_FIELD_NUMBER: builtins.int + table_name: builtins.str + """(Required)""" + def __init__( + self, + *, + table_name: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["table_name", b"table_name"] + ) -> None: ... + +global___ListPartitions = ListPartitions + +class ListViews(google.protobuf.message.Message): + """See `spark.catalog.listViews`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + DB_NAME_FIELD_NUMBER: builtins.int + PATTERN_FIELD_NUMBER: builtins.int + db_name: builtins.str + """(Optional)""" + pattern: builtins.str + """(Optional) The pattern that the view name needs to match""" + def __init__( + self, + *, + db_name: builtins.str | None = ..., + pattern: builtins.str | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_db_name", + b"_db_name", + "_pattern", + b"_pattern", + "db_name", + b"db_name", + "pattern", + b"pattern", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_db_name", + b"_db_name", + "_pattern", + b"_pattern", + "db_name", + b"db_name", + "pattern", + b"pattern", + ], + ) -> None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_db_name", b"_db_name"] + ) -> typing_extensions.Literal["db_name"] | None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_pattern", b"_pattern"] + ) -> typing_extensions.Literal["pattern"] | None: ... + +global___ListViews = ListViews + +class GetTableProperties(google.protobuf.message.Message): + """See `spark.catalog.getTableProperties`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TABLE_NAME_FIELD_NUMBER: builtins.int + table_name: builtins.str + """(Required)""" + def __init__( + self, + *, + table_name: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["table_name", b"table_name"] + ) -> None: ... + +global___GetTableProperties = GetTableProperties + +class GetCreateTableString(google.protobuf.message.Message): + """See `spark.catalog.getCreateTableString`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TABLE_NAME_FIELD_NUMBER: builtins.int + AS_SERDE_FIELD_NUMBER: builtins.int + table_name: builtins.str + """(Required)""" + as_serde: builtins.bool + def __init__( + self, + *, + table_name: builtins.str = ..., + as_serde: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal["as_serde", b"as_serde", "table_name", b"table_name"], + ) -> None: ... + +global___GetCreateTableString = GetCreateTableString + +class TruncateTable(google.protobuf.message.Message): + """See `spark.catalog.truncateTable`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TABLE_NAME_FIELD_NUMBER: builtins.int + table_name: builtins.str + """(Required)""" + def __init__( + self, + *, + table_name: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["table_name", b"table_name"] + ) -> None: ... + +global___TruncateTable = TruncateTable + +class AnalyzeTable(google.protobuf.message.Message): + """See `spark.catalog.analyzeTable`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TABLE_NAME_FIELD_NUMBER: builtins.int + NO_SCAN_FIELD_NUMBER: builtins.int + table_name: builtins.str + """(Required)""" + no_scan: builtins.bool + def __init__( + self, + *, + table_name: builtins.str = ..., + no_scan: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal["no_scan", b"no_scan", "table_name", b"table_name"], + ) -> None: ... + +global___AnalyzeTable = AnalyzeTable diff --git a/python/pyspark/sql/tests/test_catalog.py b/python/pyspark/sql/tests/test_catalog.py index 190679fb75d80..1269a1c4f34ae 100644 --- a/python/pyspark/sql/tests/test_catalog.py +++ b/python/pyspark/sql/tests/test_catalog.py @@ -14,6 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import tempfile + from pyspark import StorageLevel from pyspark.errors import AnalysisException, PySparkTypeError from pyspark.sql.types import StructType, StructField, IntegerType @@ -496,6 +498,112 @@ def test_refresh_table(self): spark.catalog.refreshTable("spark_catalog.default.my_tab") self.assertEqual(spark.table("my_tab").count(), 0) + def test_catalog_list_cached_tables(self): + spark = self.spark + t = "py_catalog_api_cached_t" + with self.table(t): + spark.sql(f"CREATE TABLE {t} (id INT) USING parquet") + spark.catalog.clearCache() + self.assertEqual(spark.catalog.listCachedTables(), []) + self.assertEqual(len(spark.sql("SHOW CACHED TABLES").collect()), 0) + spark.catalog.cacheTable(t) + cached = spark.catalog.listCachedTables() + self.assertTrue(any(t in ct.name for ct in cached)) + sql_set = {(r[0], r[1]) for r in spark.sql("SHOW CACHED TABLES").collect()} + api_set = {(ct.name, ct.storageLevel) for ct in cached} + self.assertEqual(sql_set, api_set) + spark.catalog.uncacheTable(t) + + def test_catalog_drop_table(self): + spark = self.spark + t = "py_catalog_api_drop_t" + with self.table(t): + spark.sql(f"CREATE TABLE {t} (id INT) USING parquet") + self.assertTrue(spark.catalog.tableExists(t)) + spark.catalog.dropTable(t) + self.assertFalse(spark.catalog.tableExists(t)) + + def test_catalog_drop_view(self): + spark = self.spark + v = "py_catalog_api_drop_v" + with self.view(v): + spark.sql(f"CREATE VIEW {v} AS SELECT 1 AS x") + self.assertTrue(spark.catalog.tableExists(v)) + spark.catalog.dropView(v) + self.assertFalse(spark.catalog.tableExists(v)) + + def test_catalog_create_and_drop_database(self): + spark = self.spark + db = "py_catalog_api_db" + with self.database(db): + spark.catalog.dropDatabase(db, ifExists=True, cascade=True) + self.assertFalse(spark.catalog.databaseExists(db)) + spark.catalog.createDatabase(db) + self.assertTrue(spark.catalog.databaseExists(db)) + spark.catalog.dropDatabase(db, ifExists=False, cascade=True) + self.assertFalse(spark.catalog.databaseExists(db)) + + def test_catalog_list_partitions(self): + spark = self.spark + t = "py_catalog_api_part_t" + with tempfile.TemporaryDirectory(prefix="py_catalog_part_") as td: + with self.table(t): + # LOCATION expects a string path; normalize for SQL single quotes + loc = td.replace("'", "''") + spark.sql( + f"CREATE TABLE {t} (id INT, p INT) USING parquet " + f"PARTITIONED BY (p) LOCATION '{loc}'" + ) + spark.sql(f"INSERT INTO {t} PARTITION (p = 7) SELECT 1") + parts = [p.partition for p in spark.catalog.listPartitions(t)] + self.assertTrue(any("p=7" in p for p in parts)) + + def test_catalog_list_views(self): + spark = self.spark + v = "py_catalog_api_list_v" + with self.view(v): + spark.sql(f"CREATE VIEW {v} AS SELECT 1 AS c") + names = [tv.name for tv in spark.catalog.listViews()] + self.assertIn(v, names) + + def test_catalog_get_table_properties(self): + spark = self.spark + t = "py_catalog_api_props_t" + with self.table(t): + spark.sql( + f"CREATE TABLE {t} (id INT) USING parquet " + "TBLPROPERTIES ('py_catalog_api_k' = 'py_catalog_api_v')" + ) + props = spark.catalog.getTableProperties(t) + self.assertEqual(props.get("py_catalog_api_k"), "py_catalog_api_v") + + def test_catalog_get_create_table_string(self): + spark = self.spark + t = "py_catalog_api_ddl_t" + with self.table(t): + spark.sql(f"CREATE TABLE {t} (id INT) USING parquet") + ddl = spark.catalog.getCreateTableString(t) + self.assertTrue(ddl) + self.assertIn("create", ddl.lower()) + + def test_catalog_truncate_table(self): + spark = self.spark + t = "py_catalog_api_trunc_t" + with self.table(t): + spark.sql(f"CREATE TABLE {t} (id INT) USING parquet") + spark.sql(f"INSERT INTO {t} VALUES (1), (2)") + self.assertEqual(spark.table(t).count(), 2) + spark.catalog.truncateTable(t) + self.assertEqual(spark.table(t).count(), 0) + + def test_catalog_analyze_table(self): + spark = self.spark + t = "py_catalog_api_analyze_t" + with self.table(t): + spark.sql(f"CREATE TABLE {t} (id INT) USING parquet") + spark.sql(f"INSERT INTO {t} VALUES (1)") + spark.catalog.analyzeTable(t, noScan=True) + class CatalogTests(CatalogTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/testing/sqlutils.py b/python/pyspark/testing/sqlutils.py index f36eff17822cb..ebbdc096917f7 100644 --- a/python/pyspark/testing/sqlutils.py +++ b/python/pyspark/testing/sqlutils.py @@ -128,6 +128,20 @@ def table(self, *tables): for t in tables: self.spark.sql("DROP TABLE IF EXISTS %s" % t) + @contextmanager + def view(self, *views): + """ + A convenient context manager for persistent (catalog) views. On exit, runs + ``DROP VIEW IF EXISTS`` for each name. For temporary views, use :meth:`temp_view`. + """ + assert hasattr(self, "spark"), "it should have 'spark' attribute, having a spark session." + + try: + yield + finally: + for v in views: + self.spark.sql("DROP VIEW IF EXISTS %s" % v) + @contextmanager def temp_view(self, *views): """ diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 34de788c6d464..89583c57d8e81 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -159,6 +159,7 @@ BUCKETS: 'BUCKETS'; BY: 'BY'; BYTE: 'BYTE'; CACHE: 'CACHE'; +CACHED: 'CACHED'; CALL: 'CALL'; CALLED: 'CALLED'; CASCADE: 'CASCADE'; diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index a1de1234ef317..43a537c6d89f1 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -359,6 +359,7 @@ statement | CLOSE multipartIdentifier #closeCursorStatement | EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)? (statement|setResetStatement) #explain + | SHOW CACHED TABLES #showCachedTables | SHOW TABLES ((FROM | IN) identifierReference)? (LIKE? pattern=stringLit)? #showTables | SHOW TABLE EXTENDED ((FROM | IN) ns=identifierReference)? @@ -1928,6 +1929,7 @@ ansiNonReserved | BY | BYTE | CACHE + | CACHED | CALLED | CASCADE | CATALOG @@ -2299,6 +2301,7 @@ nonReserved | BY | BYTE | CACHE + | CACHED | CALL | CALLED | CASCADE diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 0b4b50af20d43..c852f830a4e24 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -21,6 +21,7 @@ import java.util import scala.jdk.CollectionConverters._ +import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.annotation.Stable import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -34,6 +35,12 @@ import org.apache.spark.storage.StorageLevel @Stable abstract class Catalog { + private def catalogUnsupported(methodName: String): Nothing = { + throw new SparkUnsupportedOperationException( + errorClass = "UNSUPPORTED_FEATURE.CATALOG_INTERFACE_METHOD", + messageParameters = Map("methodName" -> methodName, "catalogClass" -> getClass.getName)) + } + /** * Returns the current database (namespace) in this session. * @@ -565,7 +572,7 @@ abstract class Catalog { def dropGlobalTempView(viewName: String): Boolean /** - * Recovers all the partitions in the directory of a table and update the catalog. Only works + * Recovers all the partitions in the directory of a table and updates the catalog. Only works * with a partitioned table, and not a view. * * @param tableName @@ -693,4 +700,204 @@ abstract class Catalog { * @since 3.5.0 */ def listCatalogs(pattern: String): Dataset[CatalogMetadata] + + /** + * Lists in-memory cache entries registered with an explicit name (via `CACHE TABLE`, + * `Catalog.cacheTable`, etc.). `Dataset.cache()` without a name is not listed. + * + * @return + * a dataset of `CachedTable` rows describing each named cache entry. + * @since 4.2.0 + */ + def listCachedTables(): Dataset[CachedTable] = catalogUnsupported("listCachedTables") + + /** + * Drops a persistent table. This does not remove temp views (use `dropTempView`). + * + * @param tableName + * name of the table to drop; may be qualified with catalog and database (namespace). + * @param ifExists + * if true, do not fail when the table does not exist. + * @param purge + * if true, skip moving data to a trash directory when the catalog supports it. + * @since 4.2.0 + */ + def dropTable(tableName: String, ifExists: Boolean = false, purge: Boolean = false): Unit = { + catalogUnsupported("dropTable") + } + + /** + * Drops a persistent view. + * + * @param viewName + * name of the view to drop; may be qualified with catalog and database (namespace). + * @param ifExists + * if true, do not fail when the view does not exist. + * @since 4.2.0 + */ + def dropView(viewName: String, ifExists: Boolean = false): Unit = { + catalogUnsupported("dropView") + } + + /** + * Creates a namespace (database/schema). `dbName` may be a multi-part identifier. + * + * @param dbName + * name of the namespace to create. + * @param ifNotExists + * if true, do not fail when the namespace already exists. + * @since 4.2.0 + */ + def createDatabase(dbName: String, ifNotExists: Boolean = false): Unit = { + createDatabase(dbName, ifNotExists, new util.HashMap[String, String]()) + } + + /** + * (Java-specific) Creates a namespace with optional properties (e.g. comment, location keys as + * used by `CREATE NAMESPACE`). + * + * @param dbName + * name of the namespace to create. + * @param ifNotExists + * if true, do not fail when the namespace already exists. + * @param properties + * string map of namespace properties. + * @since 4.2.0 + */ + def createDatabase( + dbName: String, + ifNotExists: Boolean, + properties: util.Map[String, String]): Unit = { + catalogUnsupported("createDatabase") + } + + /** + * (Scala-specific) Creates a namespace with optional properties map. + * + * @param dbName + * name of the namespace to create. + * @param ifNotExists + * if true, do not fail when the namespace already exists. + * @param properties + * map of namespace property keys to values. + * @since 4.2.0 + */ + def createDatabase( + dbName: String, + ifNotExists: Boolean, + properties: Map[String, String]): Unit = { + createDatabase(dbName, ifNotExists, properties.asJava) + } + + /** + * Drops a namespace. + * + * @param dbName + * name of the namespace to drop; may be qualified with catalog name. + * @param ifExists + * if true, do not fail when the namespace does not exist. + * @param cascade + * if true, also drop tables and functions in the namespace. + * @since 4.2.0 + */ + def dropDatabase(dbName: String, ifExists: Boolean = false, cascade: Boolean = false): Unit = { + catalogUnsupported("dropDatabase") + } + + /** + * Lists partition value strings for a table (same as `SHOW PARTITIONS`). + * + * @param tableName + * name of the partitioned table; may be qualified with catalog and database (namespace). + * @since 4.2.0 + */ + def listPartitions(tableName: String): Dataset[CatalogTablePartition] = { + catalogUnsupported("listPartitions") + } + + /** + * Lists views in the current namespace. + * + * @return + * a dataset of `Table` rows for each view (same shape as `listTables`). + * @since 4.2.0 + */ + def listViews(): Dataset[Table] = catalogUnsupported("listViews()") + + /** + * Lists views in the given namespace (can be catalog-qualified). + * + * @param dbName + * namespace to list views from. + * @return + * a dataset of `Table` rows for each view (same shape as `listTables`). + * @since 4.2.0 + */ + @throws[AnalysisException]("namespace does not exist") + def listViews(dbName: String): Dataset[Table] = + catalogUnsupported("listViews(String)") + + /** + * Lists views in the given namespace with a name pattern (SQL LIKE string). + * + * @param dbName + * namespace to list views from. + * @param pattern + * SQL LIKE pattern for view names. + * @return + * a dataset of `Table` rows for each matching view (same shape as `listTables`). + * @since 4.2.0 + */ + @throws[AnalysisException]("namespace does not exist") + def listViews(dbName: String, pattern: String): Dataset[Table] = { + catalogUnsupported("listViews(String, String)") + } + + /** + * Returns all table properties as a map (same as `SHOW TBLPROPERTIES`). + * + * @param tableName + * table or view name; may be qualified with catalog and database (namespace). + * @since 4.2.0 + */ + def getTableProperties(tableName: String): util.Map[String, String] = { + catalogUnsupported("getTableProperties") + } + + /** + * Returns the `SHOW CREATE TABLE` DDL string for a relation. + * + * @param tableName + * table or view name; may be qualified with catalog and database (namespace). + * @param asSerde + * if true, request Hive serde DDL when applicable. + * @since 4.2.0 + */ + def getCreateTableString(tableName: String, asSerde: Boolean = false): String = { + catalogUnsupported("getCreateTableString") + } + + /** + * Truncates a table (removes all data from the table; not supported for views). + * + * @param tableName + * name of the table to truncate; may be qualified with catalog and database (namespace). + * @since 4.2.0 + */ + def truncateTable(tableName: String): Unit = { + catalogUnsupported("truncateTable") + } + + /** + * Computes table statistics (same as `ANALYZE TABLE ... COMPUTE STATISTICS`). + * + * @param tableName + * table or view name; may be qualified with catalog and database (namespace). + * @param noScan + * if true, use `NOSCAN` mode (reuse existing column statistics where possible). + * @since 4.2.0 + */ + def analyzeTable(tableName: String, noScan: Boolean = false): Unit = { + catalogUnsupported("analyzeTable") + } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalog/interface.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalog/interface.scala index 3a3ba9d261326..da9a2dc4efdd7 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalog/interface.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalog/interface.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams // DefinedByConstructorParams for the catalog to be able to create encoders for them. /** - * A catalog in Spark, as returned by the `listCatalogs` method defined in [[Catalog]]. + * A catalog in Spark, as returned by the `listCatalogs` method defined in `Catalog`. * * @param name * name of the catalog @@ -42,7 +42,48 @@ class CatalogMetadata(val name: String, @Nullable val description: String) } /** - * A database in Spark, as returned by the `listDatabases` method defined in [[Catalog]]. + * An in-memory cached relation registered with a name, as returned by `Catalog.listCachedTables`. + * + * @param name + * table or view name used when caching (same string passed to `CACHE TABLE` / `cacheTable`). + * @param storageLevel + * storage level description of the cached data. + * @since 4.2.0 + */ +@Stable +class CachedTable(val name: String, val storageLevel: String) extends DefinedByConstructorParams { + + override def toString: String = + s"CachedTable[name='$name', storageLevel='$storageLevel']" +} + +/** + * A partition string as returned by `Catalog.listPartitions` (same form as `SHOW PARTITIONS`). + * + * @param partition + * partition description string. + * @since 4.2.0 + */ +@Stable +class CatalogTablePartition(val partition: String) extends DefinedByConstructorParams { + + override def toString: String = s"CatalogTablePartition[partition='$partition']" +} + +/** + * A table property key and value, as returned by `Catalog.getTableProperties` rows. + * + * @since 4.2.0 + */ +@Stable +class CatalogTableProperty(val key: String, val value: String) + extends DefinedByConstructorParams { + + override def toString: String = s"CatalogTableProperty[key='$key', value='$value']" +} + +/** + * A database in Spark, as returned by the `listDatabases` method defined in `Catalog`. * * @param name * name of the database. @@ -77,7 +118,7 @@ class Database( } /** - * A table in Spark, as returned by the `listTables` method in [[Catalog]]. + * A table in Spark, as returned by the `listTables` method in `Catalog`. * * @param name * name of the table. @@ -139,7 +180,7 @@ class Table( } /** - * A column in Spark, as returned by `listColumns` method in [[Catalog]]. + * A column in Spark, as returned by `listColumns` method in `Catalog`. * * @param name * name of the column. @@ -192,7 +233,7 @@ class Column( } /** - * A user-defined function in Spark, as returned by `listFunctions` method in [[Catalog]]. + * A user-defined function in Spark, as returned by `listFunctions` method in `Catalog`. * * @param name * name of the function. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index ab9b96104aac0..a03ae95397096 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -6308,6 +6308,13 @@ class AstBuilder extends DataTypeAstBuilder partitionKeys) } + /** + * Create a [[ShowCachedTables]] command. + */ + override def visitShowCachedTables(ctx: ShowCachedTablesContext): LogicalPlan = withOrigin(ctx) { + ShowCachedTables + } + /** * Create a [[RefreshTable]]. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index c0fe58596c9fd..1891985eea389 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -1653,6 +1653,18 @@ object ShowPartitions { } } +/** + * The logical plan of the SHOW CACHED TABLES command. + * + * Lists in-memory cache entries that were registered with an explicit table or view name + * (for example via `CACHE TABLE` or `Catalog.cacheTable`). + */ +case object ShowCachedTables extends LeafCommand { + override val output: Seq[Attribute] = Seq( + AttributeReference("tableName", StringType, nullable = false)(), + AttributeReference("storageLevel", StringType, nullable = false)()) +} + /** * The logical plan of the DROP VIEW command. */ diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala index 903e7edac2230..58811bffa3a62 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala @@ -209,7 +209,7 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark withConnection { conn => val metadata = conn.getMetaData // scalastyle:off line.size.limit - assert(metadata.getSQLKeywords === "ADD,AFTER,AGGREGATE,ALWAYS,ANALYZE,ANTI,ANY_VALUE,ARCHIVE,ASC,BINDING,BUCKET,BUCKETS,BYTE,CACHE,CASCADE,CATALOG,CATALOGS,CHANGE,CHANGES,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATION,COLLECTION,COLUMNS,COMMENT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONTAINS,CONTINUE,COST,DATA,DATABASE,DATABASES,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAYOFYEAR,DAYS,DBPROPERTIES,DEFINED,DEFINER,DELAY,DELIMITED,DESC,DFS,DIRECTORIES,DIRECTORY,DISTRIBUTE,DIV,DO,ELSEIF,ENFORCED,ESCAPED,EVOLUTION,EXCHANGE,EXCLUDE,EXCLUSIVE,EXIT,EXPLAIN,EXPORT,EXTEND,EXTENDED,FIELDS,FILEFORMAT,FIRST,FLOW,FOLLOWING,FORMAT,FORMATTED,FOUND,FUNCTIONS,GENERATED,GEOGRAPHY,GEOMETRY,HANDLER,HOURS,IDENTIFIED,IDENTIFIER,IF,IGNORE,ILIKE,IMMEDIATE,INCLUDE,INCLUSIVE,INCREMENT,INDEX,INDEXES,INPATH,INPUT,INPUTFORMAT,INVOKER,ITEMS,ITERATE,JSON,KEY,KEYS,LAST,LAZY,LEAVE,LEVEL,LIMIT,LINES,LIST,LOAD,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MATERIALIZED,MEASURE,METRICS,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTES,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NORELY,NULLS,OFFSET,OPTION,OPTIONS,OUTPUTFORMAT,OVERWRITE,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,PRECEDING,PRINCIPALS,PROCEDURES,PROPERTIES,PURGE,QUARTER,QUERY,RECORDREADER,RECORDWRITER,RECOVER,RECURSION,REDUCE,REFRESH,RELY,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,ROLE,ROLES,SCHEMA,SCHEMAS,SECONDS,SECURITY,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SETS,SHORT,SHOW,SINGLE,SKEWED,SORT,SORTED,SOURCE,STATISTICS,STORED,STRATIFY,STREAM,STREAMING,STRING,STRUCT,SUBSTR,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLES,TARGET,TBLPROPERTIES,TERMINATED,TIMEDIFF,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TOUCH,TRANSACTION,TRANSACTIONS,TRANSFORM,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNLOCK,UNPIVOT,UNSET,UNTIL,USE,VAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WATERMARK,WEEK,WEEKS,WHILE,X,YEARS,ZONE") + assert(metadata.getSQLKeywords === "ADD,AFTER,AGGREGATE,ALWAYS,ANALYZE,ANTI,ANY_VALUE,ARCHIVE,ASC,BINDING,BUCKET,BUCKETS,BYTE,CACHE,CACHED,CASCADE,CATALOG,CATALOGS,CHANGE,CHANGES,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATION,COLLECTION,COLUMNS,COMMENT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONTAINS,CONTINUE,COST,DATA,DATABASE,DATABASES,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAYOFYEAR,DAYS,DBPROPERTIES,DEFINED,DEFINER,DELAY,DELIMITED,DESC,DFS,DIRECTORIES,DIRECTORY,DISTRIBUTE,DIV,DO,ELSEIF,ENFORCED,ESCAPED,EVOLUTION,EXCHANGE,EXCLUDE,EXCLUSIVE,EXIT,EXPLAIN,EXPORT,EXTEND,EXTENDED,FIELDS,FILEFORMAT,FIRST,FLOW,FOLLOWING,FORMAT,FORMATTED,FOUND,FUNCTIONS,GENERATED,GEOGRAPHY,GEOMETRY,HANDLER,HOURS,IDENTIFIED,IDENTIFIER,IF,IGNORE,ILIKE,IMMEDIATE,INCLUDE,INCLUSIVE,INCREMENT,INDEX,INDEXES,INPATH,INPUT,INPUTFORMAT,INVOKER,ITEMS,ITERATE,JSON,KEY,KEYS,LAST,LAZY,LEAVE,LEVEL,LIMIT,LINES,LIST,LOAD,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MATERIALIZED,MEASURE,METRICS,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTES,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NORELY,NULLS,OFFSET,OPTION,OPTIONS,OUTPUTFORMAT,OVERWRITE,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,PRECEDING,PRINCIPALS,PROCEDURES,PROPERTIES,PURGE,QUARTER,QUERY,RECORDREADER,RECORDWRITER,RECOVER,RECURSION,REDUCE,REFRESH,RELY,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,ROLE,ROLES,SCHEMA,SCHEMAS,SECONDS,SECURITY,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SETS,SHORT,SHOW,SINGLE,SKEWED,SORT,SORTED,SOURCE,STATISTICS,STORED,STRATIFY,STREAM,STREAMING,STRING,STRUCT,SUBSTR,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLES,TARGET,TBLPROPERTIES,TERMINATED,TIMEDIFF,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TOUCH,TRANSACTION,TRANSACTIONS,TRANSFORM,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNLOCK,UNPIVOT,UNSET,UNTIL,USE,VAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WATERMARK,WEEK,WEEKS,WHILE,X,YEARS,ZONE") // scalastyle:on line.size.limit } } diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala index 40b5444b84ed3..035ba4a33bc90 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.connect import java.io.{File, FilenameFilter} +import scala.jdk.CollectionConverters._ + import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession, SQLHelper} @@ -365,4 +367,137 @@ class CatalogSuite extends ConnectFunSuite with RemoteSparkSession with SQLHelpe assert(freshColumns.contains("c3")) } } + + test("extended catalog API: listCachedTables matches SHOW CACHED TABLES") { + val t = "connect_catalog_ext_cached_t" + withTable(t) { + withTempPath { dir => + val session = spark + import session.implicits._ + Seq(1).toDF("id").write.parquet(dir.getPath) + spark.catalog.createTable(t, dir.getPath).collect() + assert(spark.catalog.listCachedTables().collect().isEmpty) + assert(spark.sql("SHOW CACHED TABLES").collect().isEmpty) + spark.catalog.cacheTable(t) + val fromApi = spark.catalog.listCachedTables().collect() + assert(fromApi.exists(_.name.contains(t))) + val fromSql = spark + .sql("SHOW CACHED TABLES") + .collect() + .map(r => (r.getString(0), r.getString(1))) + .toSet + val fromApiSet = fromApi.map(c => (c.name, c.storageLevel)).toSet + assert(fromSql === fromApiSet) + spark.catalog.uncacheTable(t) + } + } + } + + test("extended catalog API: dropTable") { + val tbl = "connect_catalog_ext_drop_t" + withTable(tbl) { + withTempPath { dir => + val session = spark + import session.implicits._ + Seq(1).toDF("id").write.parquet(dir.getPath) + spark.catalog.createTable(tbl, dir.getPath).collect() + assert(spark.catalog.tableExists(tbl)) + spark.catalog.dropTable(tbl) + assert(!spark.catalog.tableExists(tbl)) + } + } + } + + test("extended catalog API: dropView") { + val view = "connect_catalog_ext_drop_v" + withView(view) { + spark.sql(s"CREATE VIEW $view AS SELECT 1 AS x") + assert(spark.catalog.tableExists(view)) + spark.catalog.dropView(view) + assert(!spark.catalog.tableExists(view)) + } + } + + test("extended catalog API: createDatabase and dropDatabase") { + val db = "connect_catalog_ext_db" + spark.catalog.dropDatabase(db, ifExists = true, cascade = true) + assert(!spark.catalog.databaseExists(db)) + spark.catalog.createDatabase(db) + assert(spark.catalog.databaseExists(db)) + spark.catalog.dropDatabase(db, ifExists = false, cascade = true) + assert(!spark.catalog.databaseExists(db)) + } + + test("extended catalog API: listPartitions") { + val t = "connect_catalog_ext_part_t" + withTable(t) { + withTempPath { dir => + val loc = dir.toURI.toString.replace("'", "\\'") + spark.sql( + s"CREATE TABLE $t (id INT, p INT) USING parquet PARTITIONED BY (p) LOCATION '$loc'") + spark.sql(s"INSERT INTO $t PARTITION (p = 7) SELECT 1") + val parts = spark.catalog.listPartitions(t).collect().map(_.partition) + assert(parts.exists(_.contains("p=7"))) + } + } + } + + test("extended catalog API: listViews") { + val v = "connect_catalog_ext_list_v" + withView(v) { + spark.sql(s"CREATE VIEW $v AS SELECT 1 AS c") + val names = spark.catalog.listViews().collect().map(_.name) + assert(names.contains(v)) + } + } + + test("extended catalog API: getTableProperties") { + val t = "connect_catalog_ext_props_t" + withTable(t) { + spark + .sql(s"CREATE TABLE $t (id INT) USING parquet " + + "TBLPROPERTIES ('connect_catalog_ext_k' = 'connect_catalog_ext_v')") + .collect() + val props = spark.catalog.getTableProperties(t).asScala.toMap + assert(props.get("connect_catalog_ext_k").contains("connect_catalog_ext_v")) + } + } + + test("extended catalog API: getCreateTableString") { + val t = "connect_catalog_ext_ddl_t" + withTable(t) { + withTempPath { dir => + val session = spark + import session.implicits._ + Seq(1).toDF("id").write.parquet(dir.getPath) + spark.catalog.createTable(t, dir.getPath).collect() + val ddl = spark.catalog.getCreateTableString(t) + assert(ddl.nonEmpty && ddl.toLowerCase(java.util.Locale.ROOT).contains("create")) + } + } + } + + test("extended catalog API: truncateTable") { + val t = "connect_catalog_ext_trunc_t" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id INT) USING parquet").collect() + spark.sql(s"INSERT INTO $t VALUES (1), (2)").collect() + assert(spark.table(t).count() == 2) + spark.catalog.truncateTable(t) + assert(spark.table(t).count() == 0) + } + } + + test("extended catalog API: analyzeTable") { + val t = "connect_catalog_ext_analyze_t" + withTable(t) { + withTempPath { dir => + val session = spark + import session.implicits._ + Seq(1).toDF("id").write.parquet(dir.getPath) + spark.catalog.createTable(t, dir.getPath).collect() + spark.catalog.analyzeTable(t, noScan = true) + } + } + } } diff --git a/sql/connect/common/src/main/protobuf/spark/connect/catalog.proto b/sql/connect/common/src/main/protobuf/spark/connect/catalog.proto index 5b1b90b0087d0..3d3a97c0b52d1 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/catalog.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/catalog.proto @@ -55,6 +55,17 @@ message Catalog { CurrentCatalog current_catalog = 24; SetCurrentCatalog set_current_catalog = 25; ListCatalogs list_catalogs = 26; + ListCachedTables list_cached_tables = 27; + DropTable drop_table = 28; + DropView drop_view = 29; + CreateDatabase create_database = 30; + DropDatabase drop_database = 31; + ListPartitions list_partitions = 32; + ListViews list_views = 33; + GetTableProperties get_table_properties = 34; + GetCreateTableString get_create_table_string = 35; + TruncateTable truncate_table = 36; + AnalyzeTable analyze_table = 37; } } @@ -241,3 +252,77 @@ message ListCatalogs { // (Optional) The pattern that the catalog name needs to match optional string pattern = 1; } + +// See `spark.catalog.listCachedTables` +message ListCachedTables { } + +// See `spark.catalog.dropTable` +message DropTable { + // (Required) + string table_name = 1; + bool if_exists = 2; + bool purge = 3; +} + +// See `spark.catalog.dropView` +message DropView { + // (Required) + string view_name = 1; + bool if_exists = 2; +} + +// See `spark.catalog.createDatabase` +message CreateDatabase { + // (Required) + string db_name = 1; + bool if_not_exists = 2; + map properties = 3; +} + +// See `spark.catalog.dropDatabase` +message DropDatabase { + // (Required) + string db_name = 1; + bool if_exists = 2; + bool cascade = 3; +} + +// See `spark.catalog.listPartitions` +message ListPartitions { + // (Required) + string table_name = 1; +} + +// See `spark.catalog.listViews` +message ListViews { + // (Optional) + optional string db_name = 1; + // (Optional) The pattern that the view name needs to match + optional string pattern = 2; +} + +// See `spark.catalog.getTableProperties` +message GetTableProperties { + // (Required) + string table_name = 1; +} + +// See `spark.catalog.getCreateTableString` +message GetCreateTableString { + // (Required) + string table_name = 1; + bool as_serde = 2; +} + +// See `spark.catalog.truncateTable` +message TruncateTable { + // (Required) + string table_name = 1; +} + +// See `spark.catalog.analyzeTable` +message AnalyzeTable { + // (Required) + string table_name = 1; + bool no_scan = 2; +} diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Catalog.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Catalog.scala index 6850ffd122608..4e9316cb69dca 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Catalog.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Catalog.scala @@ -17,9 +17,13 @@ package org.apache.spark.sql.connect -import org.apache.spark.sql.AnalysisException +import java.util + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.{AnalysisException, Dataset} import org.apache.spark.sql.catalog -import org.apache.spark.sql.catalog.{CatalogMetadata, Column, Database, Function, Table} +import org.apache.spark.sql.catalog.{CachedTable, CatalogMetadata, CatalogTablePartition, Column, Database, Function, Table} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{PrimitiveBooleanEncoder, StringEncoder} @@ -704,9 +708,221 @@ class Catalog(sparkSession: SparkSession) extends catalog.Catalog { .newDataset(Catalog.catalogEncoder) { builder => builder.getCatalogBuilder.getListCatalogsBuilder.setPattern(pattern) } + + /** + * Lists in-memory cache entries registered with an explicit name (via `CACHE TABLE`, + * `Catalog.cacheTable`, etc.). `Dataset.cache()` without a name is not listed. + * + * @since 4.2.0 + */ + override def listCachedTables(): Dataset[CachedTable] = { + sparkSession.newDataset(Catalog.cachedTableEncoder) { builder => + builder.getCatalogBuilder.getListCachedTablesBuilder + } + } + + /** + * Drops a persistent table. This does not remove temp views (use `dropTempView`). + * + * @param tableName + * qualified or unqualified table name + * @since 4.2.0 + */ + override def dropTable(tableName: String, ifExists: Boolean, purge: Boolean): Unit = { + sparkSession.execute { builder => + builder.getCatalogBuilder.getDropTableBuilder + .setTableName(tableName) + .setIfExists(ifExists) + .setPurge(purge) + } + } + + /** + * Drops a persistent view. + * + * @param viewName + * qualified or unqualified view name + * @since 4.2.0 + */ + override def dropView(viewName: String, ifExists: Boolean): Unit = { + sparkSession.execute { builder => + builder.getCatalogBuilder.getDropViewBuilder + .setViewName(viewName) + .setIfExists(ifExists) + } + } + + /** + * Creates a namespace (database/schema). `dbName` may be a multi-part identifier. + * + * @param dbName + * namespace to create + * @param properties + * optional properties (e.g. comment, location keys as used by `CREATE NAMESPACE`) + * @since 4.2.0 + */ + override def createDatabase( + dbName: String, + ifNotExists: Boolean, + properties: util.Map[String, String]): Unit = { + sparkSession.execute { builder => + val b = builder.getCatalogBuilder.getCreateDatabaseBuilder + .setDbName(dbName) + .setIfNotExists(ifNotExists) + properties.asScala.foreach { case (k, v) => b.putProperties(k, v) } + } + } + + /** + * Drops a namespace. + * + * @param dbName + * namespace to drop + * @since 4.2.0 + */ + override def dropDatabase(dbName: String, ifExists: Boolean, cascade: Boolean): Unit = { + sparkSession.execute { builder => + builder.getCatalogBuilder.getDropDatabaseBuilder + .setDbName(dbName) + .setIfExists(ifExists) + .setCascade(cascade) + } + } + + /** + * Lists partition value strings for a table (same as `SHOW PARTITIONS`). + * + * @param tableName + * qualified or unqualified table name + * @since 4.2.0 + */ + override def listPartitions(tableName: String): Dataset[CatalogTablePartition] = { + sparkSession.newDataset(Catalog.catalogTablePartitionEncoder) { builder => + builder.getCatalogBuilder.getListPartitionsBuilder.setTableName(tableName) + } + } + + /** + * Lists views in the current namespace. + * + * @since 4.2.0 + */ + override def listViews(): Dataset[Table] = { + sparkSession.newDataset(Catalog.tableEncoder) { builder => + builder.getCatalogBuilder.getListViewsBuilder + } + } + + /** + * Lists views in the given namespace (can be catalog-qualified). + * + * @param dbName + * namespace to list views from + * @since 4.2.0 + */ + @throws[AnalysisException]("namespace does not exist") + override def listViews(dbName: String): Dataset[Table] = { + sparkSession.newDataset(Catalog.tableEncoder) { builder => + builder.getCatalogBuilder.getListViewsBuilder.setDbName(dbName) + } + } + + /** + * Lists views in the given namespace with a name pattern (SQL LIKE string). + * + * @param dbName + * namespace to list views from + * @param pattern + * SQL LIKE pattern for view names + * @since 4.2.0 + */ + @throws[AnalysisException]("namespace does not exist") + override def listViews(dbName: String, pattern: String): Dataset[Table] = { + sparkSession.newDataset(Catalog.tableEncoder) { builder => + builder.getCatalogBuilder.getListViewsBuilder + .setDbName(dbName) + .setPattern(pattern) + } + } + + /** + * Returns all table properties as a map (same as `SHOW TBLPROPERTIES`). + * + * @param tableName + * qualified or unqualified table or view name + * @since 4.2.0 + */ + override def getTableProperties(tableName: String): util.Map[String, String] = { + val df = sparkSession.newDataFrame { builder => + builder.getCatalogBuilder.getGetTablePropertiesBuilder.setTableName(tableName) + } + val m = new util.HashMap[String, String]() + df.collect().foreach { row => + m.put(row.getString(0), row.getString(1)) + } + m + } + + /** + * Returns the `SHOW CREATE TABLE` DDL string for a relation. + * + * @param tableName + * qualified or unqualified table or view name + * @param asSerde + * if true, request Hive serde DDL when applicable + * @since 4.2.0 + */ + override def getCreateTableString(tableName: String, asSerde: Boolean): String = { + sparkSession + .newDataset(StringEncoder) { builder => + builder.getCatalogBuilder.getGetCreateTableStringBuilder + .setTableName(tableName) + .setAsSerde(asSerde) + } + .head() + } + + /** + * Truncates a table (removes all data from the table; not supported for views). + * + * @param tableName + * qualified or unqualified table name + * @since 4.2.0 + */ + override def truncateTable(tableName: String): Unit = { + sparkSession.execute { builder => + builder.getCatalogBuilder.getTruncateTableBuilder.setTableName(tableName) + } + } + + /** + * Computes table statistics (same as `ANALYZE TABLE ... COMPUTE STATISTICS`). + * + * @param tableName + * qualified or unqualified table name + * @param noScan + * if true, use `NOSCAN` mode + * @since 4.2.0 + */ + override def analyzeTable(tableName: String, noScan: Boolean): Unit = { + sparkSession.execute { builder => + builder.getCatalogBuilder.getAnalyzeTableBuilder + .setTableName(tableName) + .setNoScan(noScan) + } + } } private object Catalog { + private val cachedTableEncoder: AgnosticEncoder[CachedTable] = ScalaReflection + .encoderFor(ScalaReflection.localTypeOf[CachedTable]) + .asInstanceOf[AgnosticEncoder[CachedTable]] + + private val catalogTablePartitionEncoder: AgnosticEncoder[CatalogTablePartition] = + ScalaReflection + .encoderFor(ScalaReflection.localTypeOf[CatalogTablePartition]) + .asInstanceOf[AgnosticEncoder[CatalogTablePartition]] + private val databaseEncoder: AgnosticEncoder[Database] = ScalaReflection .encoderFor(ScalaReflection.localTypeOf[Database]) .asInstanceOf[AgnosticEncoder[Database]] diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 23403381ed8b2..37bcf995ee16d 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -83,6 +83,7 @@ import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode, StatefulPr import org.apache.spark.sql.types._ import org.apache.spark.sql.util.{ArrowUtils, CaseInsensitiveStringMap} import org.apache.spark.storage.CacheId +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -313,6 +314,25 @@ class SparkConnectPlanner( transformSetCurrentCatalog(catalog.getSetCurrentCatalog) case proto.Catalog.CatTypeCase.LIST_CATALOGS => transformListCatalogs(catalog.getListCatalogs) + case proto.Catalog.CatTypeCase.LIST_CACHED_TABLES => + transformListCachedTables() + case proto.Catalog.CatTypeCase.DROP_TABLE => transformDropTable(catalog.getDropTable) + case proto.Catalog.CatTypeCase.DROP_VIEW => transformDropView(catalog.getDropView) + case proto.Catalog.CatTypeCase.CREATE_DATABASE => + transformCreateDatabase(catalog.getCreateDatabase) + case proto.Catalog.CatTypeCase.DROP_DATABASE => + transformDropDatabase(catalog.getDropDatabase) + case proto.Catalog.CatTypeCase.LIST_PARTITIONS => + transformListPartitions(catalog.getListPartitions) + case proto.Catalog.CatTypeCase.LIST_VIEWS => transformListViews(catalog.getListViews) + case proto.Catalog.CatTypeCase.GET_TABLE_PROPERTIES => + transformGetTableProperties(catalog.getGetTableProperties) + case proto.Catalog.CatTypeCase.GET_CREATE_TABLE_STRING => + transformGetCreateTableString(catalog.getGetCreateTableString) + case proto.Catalog.CatTypeCase.TRUNCATE_TABLE => + transformTruncateTable(catalog.getTruncateTable) + case proto.Catalog.CatTypeCase.ANALYZE_TABLE => + transformAnalyzeTable(catalog.getAnalyzeTable) case other => throw InvalidInputErrors.invalidOneOfField(other, catalog.getDescriptorForType) } @@ -4254,6 +4274,79 @@ class SparkConnectPlanner( } } + private def transformListCachedTables(): LogicalPlan = { + session.catalog.listCachedTables().logicalPlan + } + + private def transformDropTable(p: proto.DropTable): LogicalPlan = { + session.catalog.dropTable(p.getTableName, p.getIfExists, p.getPurge) + emptyLocalRelation + } + + private def transformDropView(p: proto.DropView): LogicalPlan = { + session.catalog.dropView(p.getViewName, p.getIfExists) + emptyLocalRelation + } + + private def transformCreateDatabase(p: proto.CreateDatabase): LogicalPlan = { + val jmap = new java.util.HashMap[String, String]() + p.getPropertiesMap.asScala.foreach { case (k, v) => jmap.put(k, v) } + session.catalog.createDatabase(p.getDbName, p.getIfNotExists, jmap) + emptyLocalRelation + } + + private def transformDropDatabase(p: proto.DropDatabase): LogicalPlan = { + session.catalog.dropDatabase(p.getDbName, p.getIfExists, p.getCascade) + emptyLocalRelation + } + + private def transformListPartitions(p: proto.ListPartitions): LogicalPlan = { + session.catalog.listPartitions(p.getTableName).logicalPlan + } + + private def transformListViews(p: proto.ListViews): LogicalPlan = { + if (p.hasDbName) { + if (p.hasPattern) { + session.catalog.listViews(p.getDbName, p.getPattern).logicalPlan + } else { + session.catalog.listViews(p.getDbName).logicalPlan + } + } else if (p.hasPattern) { + val currentDatabase = session.catalog.currentDatabase + session.catalog.listViews(currentDatabase, p.getPattern).logicalPlan + } else { + session.catalog.listViews().logicalPlan + } + } + + private def transformGetTableProperties(p: proto.GetTableProperties): LogicalPlan = { + val props = session.catalog.getTableProperties(p.getTableName).asScala + val attrs = Seq( + AttributeReference("key", StringType, nullable = false)(), + AttributeReference("value", StringType, nullable = true)()) + val rows = props.map { case (k, v) => + InternalRow(UTF8String.fromString(k), UTF8String.fromString(v)) + }.toSeq + LocalRelation(attrs, rows) + } + + private def transformGetCreateTableString(p: proto.GetCreateTableString): LogicalPlan = { + session + .createDataset(session.catalog.getCreateTableString(p.getTableName, p.getAsSerde) :: Nil)( + Encoders.STRING) + .logicalPlan + } + + private def transformTruncateTable(p: proto.TruncateTable): LogicalPlan = { + session.catalog.truncateTable(p.getTableName) + emptyLocalRelation + } + + private def transformAnalyzeTable(p: proto.AnalyzeTable): LogicalPlan = { + session.catalog.analyzeTable(p.getTableName, p.getNoScan) + emptyLocalRelation + } + private def transformSubqueryExpression( getSubqueryExpression: proto.SubqueryExpression): Expression = { val planId = getSubqueryExpression.getPlanId diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala index c84f3c15ed36b..fb0d58c76ca57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.classic +import java.util + +import scala.jdk.CollectionConverters._ import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal @@ -24,7 +27,15 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CATALOG_NAME, DATABASE_NAME, TABLE_NAME} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog -import org.apache.spark.sql.catalog.{CatalogMetadata, Column, Database, Function, Table} +import org.apache.spark.sql.catalog.{ + CachedTable, + CatalogMetadata, + CatalogTablePartition, + Column, + Database, + Function, + Table +} import org.apache.spark.sql.catalyst.DefinedByConstructorParams import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis._ @@ -32,14 +43,39 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, LocalRelation, LogicalPlan, OptionList, RecoverPartitions, ShowFunctions, ShowTables, UnresolvedTableSpec, View} +import org.apache.spark.sql.catalyst.plans.logical.{ + AnalyzeTable, + ColumnDefinition, + CreateNamespace, + CreateTable, + DropNamespace, + DropTable, + DropView, + LocalRelation, + LogicalPlan, + OptionList, + RecoverPartitions, + ShowCreateTable, + ShowFunctions, + ShowPartitions, + ShowTableProperties, + ShowTables, + ShowViews, + TruncateTable, + UnresolvedTableSpec, + View +} import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.{CatalogManager, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.{CatalogHelper, MultipartIdentifierHelper, NamespaceHelper, TransformHelper} import org.apache.spark.sql.connector.catalog.CatalogV2Util.v2ColumnsToStructType import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.execution.command.{ShowNamespacesCommand, ShowTablesCommand} +import org.apache.spark.sql.execution.command.{ + ShowNamespacesCommand, + ShowTablesCommand, + ShowViewsCommand +} import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation @@ -952,6 +988,120 @@ class Catalog(sparkSession: SparkSession) extends catalog.Catalog with Logging { Catalog.makeDataset(catalogs.map(name => makeCatalog(name)), sparkSession) } + override def listCachedTables(): Dataset[CachedTable] = { + val cached = sparkSession.sharedState.cacheManager.listNamedCachedTables().map { case (n, sl) => + new CachedTable(n, sl.description) + } + Catalog.makeDataset(cached, sparkSession) + } + + override def dropTable(tableName: String, ifExists: Boolean, purge: Boolean): Unit = { + val plan = DropTable( + UnresolvedIdentifier(parseIdent(tableName), allowTemp = true), + ifExists, + purge) + sparkSession.sessionState.executePlan(plan).toRdd + } + + override def dropView(viewName: String, ifExists: Boolean): Unit = { + val plan = DropView( + UnresolvedIdentifier(parseIdent(viewName), allowTemp = true), + ifExists) + sparkSession.sessionState.executePlan(plan).toRdd + } + + override def createDatabase( + dbName: String, + ifNotExists: Boolean, + properties: util.Map[String, String]): Unit = { + val props = properties.asScala.toMap + val plan = CreateNamespace(UnresolvedNamespace(parseIdent(dbName)), ifNotExists, props) + sparkSession.sessionState.executePlan(plan).toRdd + } + + override def dropDatabase(dbName: String, ifExists: Boolean, cascade: Boolean): Unit = { + val plan = DropNamespace(UnresolvedNamespace(parseIdent(dbName)), ifExists, cascade) + sparkSession.sessionState.executePlan(plan).toRdd + } + + override def listPartitions(tableName: String): Dataset[CatalogTablePartition] = { + val plan = ShowPartitions( + UnresolvedTable(toTableIdent(tableName), "Catalog.listPartitions"), + None) + val partitions = sparkSession.sessionState.executePlan(plan).toRdd.collect().map { row => + new CatalogTablePartition(row.getString(0)) + } + Catalog.makeDataset(partitions.toImmutableArraySeq, sparkSession) + } + + override def listViews(): Dataset[Table] = { + listViewsInternal(CurrentNamespace, None) + } + + @throws[AnalysisException]("database does not exist") + override def listViews(dbName: String): Dataset[Table] = { + listViewsInternal(UnresolvedNamespace(resolveNamespace(dbName)), None) + } + + @throws[AnalysisException]("database does not exist") + override def listViews(dbName: String, pattern: String): Dataset[Table] = { + listViewsInternal(UnresolvedNamespace(resolveNamespace(dbName)), Some(pattern)) + } + + private def listViewsInternal(ns: LogicalPlan, pattern: Option[String]): Dataset[Table] = { + val plan = ShowViews(ns, pattern) + makeViewsDataset(plan) + } + + private def makeViewsDataset(plan: ShowViews): Dataset[Table] = { + val qe = sparkSession.sessionState.executePlan(plan) + val catalog = qe.analyzed.collectFirst { + case ShowViews(r: ResolvedNamespace, _, _) => r.catalog + case _: ShowViewsCommand => + sparkSession.sessionState.catalogManager.v2SessionCatalog + }.get + val tables = qe.toRdd.collect().flatMap { row => resolveTable(row, catalog.name()) } + Catalog.makeDataset(tables.toImmutableArraySeq, sparkSession) + } + + override def getTableProperties(tableName: String): util.Map[String, String] = { + val plan = ShowTableProperties( + UnresolvedTableOrView( + toTableIdent(tableName), + "Catalog.getTableProperties", + allowTempView = true), + None) + val m = new util.HashMap[String, String]() + sparkSession.sessionState.executePlan(plan).toRdd.collect().foreach { row => + m.put(row.getString(0), row.getString(1)) + } + m + } + + override def getCreateTableString(tableName: String, asSerde: Boolean): String = { + val plan = ShowCreateTable( + UnresolvedTableOrView( + toTableIdent(tableName), + "Catalog.getCreateTableString", + allowTempView = true), + asSerde) + val rows = sparkSession.sessionState.executePlan(plan).toRdd.collect() + if (rows.isEmpty) "" else rows(0).getString(0) + } + + override def truncateTable(tableName: String): Unit = { + val plan = TruncateTable(UnresolvedTable(toTableIdent(tableName), "Catalog.truncateTable")) + sparkSession.sessionState.executePlan(plan).toRdd + } + + override def analyzeTable(tableName: String, noScan: Boolean): Unit = { + val plan = AnalyzeTable( + UnresolvedTable(toTableIdent(tableName), "Catalog.analyzeTable"), + partitionSpec = Map.empty, + noScan = noScan) + sparkSession.sessionState.executePlan(plan).toRdd + } + private def makeCatalog(name: String): CatalogMetadata = { new CatalogMetadata( name = name, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 3f92f24156d3c..56faf6fcc727c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -94,6 +94,18 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { cachedData.size } + /** + * Returns cache entries that were registered with an explicit table/view name (e.g. + * `CACHE TABLE` / `Catalog.cacheTable`). Anonymous `Dataset.cache()` entries are omitted. + */ + private[sql] def listNamedCachedTables(): Seq[(String, StorageLevel)] = this.synchronized { + cachedData.flatMap { cd => + cd.cachedRepresentation.cacheBuilder.tableName.map { n => + (n, cd.cachedRepresentation.cacheBuilder.storageLevel) + } + } + } + // Test-only def cacheQuery(query: Dataset[_]): Unit = { cacheQuery(query, tableName = None, storageLevel = MEMORY_AND_DISK) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5c393b1db227e..f11dcbd1e7c1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -903,6 +903,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case d: DataWritingCommand => DataWritingCommandExec(d, planLater(d.query)) :: Nil + case ShowCachedTables => + ExecutedCommandExec(ShowCachedTablesCommand) :: Nil case r: RunnableCommand => ExecutedCommandExec(r) :: Nil case MemoryPlan(sink, output) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 9c3ac9ef74191..ed46b91cbde53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.ShowCachedTables /** * Clear all cached data from the in-memory cache. @@ -29,3 +31,17 @@ case object ClearCacheCommand extends LeafRunnableCommand { Seq.empty[Row] } } + +/** + * The command for `SHOW CACHED TABLES`. + */ +case object ShowCachedTablesCommand extends LeafRunnableCommand { + + override val output: Seq[Attribute] = ShowCachedTables.output + + override def run(sparkSession: SparkSession): Seq[Row] = { + sparkSession.sharedState.cacheManager.listNamedCachedTables().map { case (name, level) => + Row(name, level.description) + } + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out index b7491bfc93dba..69599411d523f 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out @@ -35,6 +35,7 @@ BUCKETS false BY false BYTE false CACHE false +CACHED false CALL true CALLED false CASCADE false diff --git a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out index ee43431e5efff..4f09bd2e3266d 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out @@ -35,6 +35,7 @@ BUCKETS false BY false BYTE false CACHE false +CACHED false CALL false CALLED false CASCADE false diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out index ee43431e5efff..4f09bd2e3266d 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out @@ -35,6 +35,7 @@ BUCKETS false BY false BYTE false CACHE false +CACHED false CALL false CALLED false CASCADE false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 9db7ae2e2824c..80684de7a5bc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -48,6 +48,12 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { ShowCurrentNamespaceCommand()) } + test("SHOW CACHED TABLES") { + comparePlans( + parser.parsePlan("SHOW CACHED TABLES"), + ShowCachedTables) + } + test("insert overwrite directory") { val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a" parser.parsePlan(v1) match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index ebfffc14b0144..8fe0e4e4f67e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.internal import java.io.File +import scala.jdk.CollectionConverters._ + +import org.apache.hadoop.fs.{FileSystem, Path} import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{AnalysisException, DataFrame} @@ -1136,6 +1139,128 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf } } + test("catalog API: SHOW CACHED TABLES and listCachedTables") { + val t = "catalog_api_ext_cached_t" + spark.sql(s"DROP TABLE IF EXISTS $t") + spark.sql(s"CREATE TABLE $t (id INT) USING parquet") + try { + assert(spark.catalog.listCachedTables().collect().isEmpty) + assert(spark.sql("SHOW CACHED TABLES").collect().isEmpty) + spark.catalog.cacheTable(t) + val fromApi = spark.catalog.listCachedTables().collect() + assert(fromApi.exists(_.name.contains(t))) + val fromSql = spark + .sql("SHOW CACHED TABLES") + .collect() + .map(r => (r.getString(0), r.getString(1))) + .toSet + val fromApiSet = fromApi.map(c => (c.name, c.storageLevel)).toSet + assert(fromSql === fromApiSet) + } finally { + spark.catalog.uncacheTable(t) + spark.catalog.dropTable(t, ifExists = true) + } + } + + test("catalog API: dropTable") { + val t = "catalog_api_ext_drop_t" + spark.sql(s"DROP TABLE IF EXISTS $t") + spark.sql(s"CREATE TABLE $t (id INT) USING parquet") + assert(spark.catalog.tableExists(t)) + spark.catalog.dropTable(t) + assert(!spark.catalog.tableExists(t)) + } + + test("catalog API: dropView") { + val v = "catalog_api_ext_drop_v" + spark.sql(s"DROP VIEW IF EXISTS $v") + spark.sql(s"CREATE VIEW $v AS SELECT 1 AS x") + assert(spark.catalog.tableExists(v)) + spark.catalog.dropView(v) + assert(!spark.catalog.tableExists(v)) + } + + test("catalog API: createDatabase and dropDatabase") { + val db = "catalog_api_ext_db" + spark.catalog.dropDatabase(db, ifExists = true, cascade = true) + assert(!spark.catalog.databaseExists(db)) + spark.catalog.createDatabase(db) + assert(spark.catalog.databaseExists(db)) + spark.catalog.dropDatabase(db, ifExists = false, cascade = true) + assert(!spark.catalog.databaseExists(db)) + } + + test("catalog API: listPartitions") { + val t = "catalog_api_ext_part_t" + withTempPath { dir => + spark.sql(s"DROP TABLE IF EXISTS $t") + val loc = dir.toURI.toString.replace("'", "\\'") + spark.sql( + s"CREATE TABLE $t (id INT, p INT) USING parquet PARTITIONED BY (p) LOCATION '$loc'") + spark.sql(s"INSERT INTO $t PARTITION (p = 7) SELECT 1") + val parts = spark.catalog.listPartitions(t).collect().map(_.partition) + assert(parts.exists(_.contains("p=7"))) + spark.catalog.dropTable(t) + } + } + + test("catalog API: listViews") { + val v = "catalog_api_ext_list_v" + spark.sql(s"DROP VIEW IF EXISTS $v") + spark.sql(s"CREATE VIEW $v AS SELECT 1 AS c") + val names = spark.catalog.listViews().collect().map(_.name) + assert(names.contains(v)) + spark.catalog.dropView(v) + } + + test("catalog API: getTableProperties") { + val t = "catalog_api_ext_props_t" + spark.sql(s"DROP TABLE IF EXISTS $t") + spark.sql( + s"CREATE TABLE $t (id INT) USING parquet " + + "TBLPROPERTIES ('catalog_api_ext_k' = 'catalog_api_ext_v')") + val props = spark.catalog.getTableProperties(t).asScala.toMap + assert(props.get("catalog_api_ext_k").contains("catalog_api_ext_v")) + spark.catalog.dropTable(t) + } + + test("catalog API: getCreateTableString") { + val t = "catalog_api_ext_ddl_t" + spark.sql(s"DROP TABLE IF EXISTS $t") + spark.sql(s"CREATE TABLE $t (id INT) USING parquet") + val ddl = spark.catalog.getCreateTableString(t) + assert(ddl.nonEmpty && ddl.toLowerCase(java.util.Locale.ROOT).contains("create")) + spark.catalog.dropTable(t) + } + + test("catalog API: truncateTable") { + val t = "catalog_api_ext_trunc_t" + spark.sql(s"DROP TABLE IF EXISTS $t") + spark.sql(s"CREATE TABLE $t (id INT) USING parquet") + spark.sql(s"INSERT INTO $t VALUES (1), (2)") + val ident = TableIdentifier(t, Some(spark.catalog.currentDatabase)) + def parquetCount(path: Path, fs: FileSystem): Int = + fs.listStatus(path).count(_.getPath.getName.endsWith(".parquet")) + val loc = sessionCatalog.getTableMetadata(ident).storage.locationUri + assert(loc.isDefined) + val tablePath = new Path(loc.get) + val fs = tablePath.getFileSystem(spark.sessionState.newHadoopConf()) + assert(parquetCount(tablePath, fs) > 0) + spark.catalog.truncateTable(t) + assert(parquetCount(tablePath, fs) == 0) + // Avoid catalog.dropTable: DropTableCommand may scan the table during uncacheQuery after data + // was removed. afterEach sessionCatalog.reset() drops the table. + } + + test("catalog API: analyzeTable") { + val t = "catalog_api_ext_analyze_t" + spark.sql(s"DROP TABLE IF EXISTS $t") + spark.sql(s"CREATE TABLE $t (id INT) USING parquet") + spark.sql(s"INSERT INTO $t VALUES (1)") + spark.catalog.analyzeTable(t, noScan = true) + spark.catalog.dropTable(t) + } + private def getConstructorParameterValues(obj: DefinedByConstructorParams): Seq[AnyRef] = { ScalaReflection.getConstructorParameterNames(obj.getClass).map { name => obj.getClass.getMethod(name).invoke(obj) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index e864b026b7cd6..29b6c766bfd6d 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -214,7 +214,7 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { val sessionHandle = client.openSession(user, "") val infoValue = client.getInfo(sessionHandle, GetInfoType.CLI_ODBC_KEYWORDS) // scalastyle:off line.size.limit - assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,ASENSITIVE,AT,ATOMIC,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHANGES,CHAR,CHARACTER,CHECK,CLEAR,CLOSE,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONDITION,CONSTRAINT,CONTAINS,CONTINUE,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,CURSOR,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELAY,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,ELSEIF,END,ENFORCED,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXCLUSIVE,EXECUTE,EXISTS,EXIT,EXPLAIN,EXPORT,EXTEND,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FLOW,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FOUND,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GEOGRAPHY,GEOMETRY,GLOBAL,GRANT,GROUP,GROUPING,HANDLER,HAVING,HOUR,HOURS,IDENTIFIED,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCLUSIVE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSENSITIVE,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,JSON,KEY,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LEVEL,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MATERIALIZED,MAX,MEASURE,MERGE,METRICS,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NEXT,NO,NONE,NORELY,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPEN,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROCEDURE,PROCEDURES,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READ,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,RECURSION,RECURSIVE,REDUCE,REFERENCES,REFRESH,RELY,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,SQLEXCEPTION,SQLSTATE,START,STATISTICS,STORED,STRATIFY,STREAM,STREAMING,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUE,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WATERMARK,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,WITHOUT,X,YEAR,YEARS,ZONE") + assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,ASENSITIVE,AT,ATOMIC,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CACHED,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHANGES,CHAR,CHARACTER,CHECK,CLEAR,CLOSE,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONDITION,CONSTRAINT,CONTAINS,CONTINUE,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,CURSOR,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELAY,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,ELSEIF,END,ENFORCED,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXCLUSIVE,EXECUTE,EXISTS,EXIT,EXPLAIN,EXPORT,EXTEND,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FLOW,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FOUND,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GEOGRAPHY,GEOMETRY,GLOBAL,GRANT,GROUP,GROUPING,HANDLER,HAVING,HOUR,HOURS,IDENTIFIED,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCLUSIVE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSENSITIVE,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,JSON,KEY,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LEVEL,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MATERIALIZED,MAX,MEASURE,MERGE,METRICS,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NEXT,NO,NONE,NORELY,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPEN,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROCEDURE,PROCEDURES,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READ,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,RECURSION,RECURSIVE,REDUCE,REFERENCES,REFRESH,RELY,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,SQLEXCEPTION,SQLSTATE,START,STATISTICS,STORED,STRATIFY,STREAM,STREAMING,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUE,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WATERMARK,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,WITHOUT,X,YEAR,YEARS,ZONE") // scalastyle:on line.size.limit } }