diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala index d646fad00c075..2e36749371df8 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala @@ -39,10 +39,10 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper { assert(dbs.length == 2) assert(dbs.map(_.name) sameElements Array(db, currentDb)) assert(dbs.map(_.catalog).distinct sameElements Array("spark_catalog")) - var databasesWithPattern = spark.catalog.listDatabases("def*").collect().sortBy(_.name) + var databasesWithPattern = spark.catalog.listDatabases("def%").collect().sortBy(_.name) assert(databasesWithPattern.length == 1) assert(databasesWithPattern.map(_.name) sameElements Array(currentDb)) - databasesWithPattern = spark.catalog.listDatabases("def2*").collect().sortBy(_.name) + databasesWithPattern = spark.catalog.listDatabases("def2%").collect().sortBy(_.name) assert(databasesWithPattern.length == 0) val database = spark.catalog.getDatabase(db) assert(database.name == db) @@ -75,10 +75,10 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper { val catalogsAfterChange = spark.catalog.listCatalogs().collect() assert(catalogsAfterChange.length == 2) assert(catalogsAfterChange.map(_.name).toSet == Set("testcat", "spark_catalog")) - var catalogsWithPattern = spark.catalog.listCatalogs("spark*").collect() + var catalogsWithPattern = spark.catalog.listCatalogs("spark%").collect() assert(catalogsWithPattern.length == 1) assert(catalogsWithPattern.map(_.name) sameElements Array("spark_catalog")) - catalogsWithPattern = spark.catalog.listCatalogs("hive*").collect() + catalogsWithPattern = spark.catalog.listCatalogs("hive%").collect() assert(catalogsWithPattern.length == 0) } finally { spark.catalog.setCurrentCatalog(currentCatalog) @@ -128,12 +128,12 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper { jsonTableName)) assert( spark.catalog - .listTables(spark.catalog.currentDatabase, "par*") + .listTables(spark.catalog.currentDatabase, "par%") .collect() .map(_.name) .toSet == Set(parquetTableName)) assert( - spark.catalog.listTables(spark.catalog.currentDatabase, "txt*").collect().isEmpty) + spark.catalog.listTables(spark.catalog.currentDatabase, "txt%").collect().isEmpty) } assert(spark.catalog.tableExists(parquetTableName)) assert(!spark.catalog.tableExists(orcTableName)) @@ -212,11 +212,11 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper { }.getMessage assert(message.contains("UNRESOLVED_ROUTINE")) - val functionsWithPattern1 = spark.catalog.listFunctions(dbName, "to*").collect() + val functionsWithPattern1 = spark.catalog.listFunctions(dbName, "to%").collect() assert(functionsWithPattern1.nonEmpty) assert(functionsWithPattern1.exists(f => f.name == "to_date")) val functionsWithPattern2 = - spark.catalog.listFunctions(dbName, "*not_existing_func*").collect() + spark.catalog.listFunctions(dbName, "%not_existing_func%").collect() assert(functionsWithPattern2.isEmpty) } diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md index fef9722a444f8..d3754b303f3b3 100644 --- a/docs/sql-ref-syntax-aux-show-tables.md +++ b/docs/sql-ref-syntax-aux-show-tables.md @@ -40,12 +40,21 @@ SHOW TABLES [ { FROM | IN } database_name ] [ LIKE regex_pattern ] * **regex_pattern** - Specifies the regular expression pattern that is used to filter out unwanted tables. + Specifies the regular expression pattern that is used to filter out unwanted tables. + 1. After Version 4.0 + * Same as SQL type `like` expressions, `%` for any character(s), and `_` for a single character. + * Examples are `'employees'`, `'emp%'`, `'emplo_ees'`, all of which will match the database named `'employees'`. + * **Note** + * The `OR` syntax represented by `|` is no longer supported by default. + * You can restore the semantics supported before version 4 by setting `spark.sql.legacy.useVerticalBarAndStarAsWildcardsInLikePattern` to true. + + 1. Before Version 4.0 * Except for `*` and `|` character, the pattern works like a regular expression. * `*` alone matches 0 or more characters and `|` is used to separate multiple different regular expressions, any of which can match. * The leading and trailing blanks are trimmed in the input pattern before processing. The pattern match is case-insensitive. + * Examples are `'employees'`, `'emp*'`, `'emp*|*ees'`, all of which will match the database named `'employees'`. ### Examples @@ -79,7 +88,7 @@ SHOW TABLES IN userdb; +--------+---------+-----------+ -- List all tables from default database matching the pattern `sam*` -SHOW TABLES FROM default LIKE 'sam*'; +SHOW TABLES FROM default LIKE 'sam%'; +--------+---------+-----------+ |database|tableName|isTemporary| +--------+---------+-----------+ @@ -88,6 +97,7 @@ SHOW TABLES FROM default LIKE 'sam*'; +--------+---------+-----------+ -- List all tables matching the pattern `sam*|suj` +Note: After Version 4.0, the `OR` syntax represented by `|` is no longer supported by default. SHOW TABLES LIKE 'sam*|suj'; +--------+---------+-----------+ |database|tableName|isTemporary| diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index d70bd89baedab..97123ad5da025 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -142,10 +142,10 @@ def listCatalogs(self, pattern: Optional[str] = None) -> List[CatalogMetadata]: >>> spark.catalog.listCatalogs() [CatalogMetadata(name='spark_catalog', description=None)] - >>> spark.catalog.listCatalogs("spark*") + >>> spark.catalog.listCatalogs("spark%") [CatalogMetadata(name='spark_catalog', description=None)] - >>> spark.catalog.listCatalogs("hive*") + >>> spark.catalog.listCatalogs("hive%") [] """ if pattern is None: @@ -213,10 +213,10 @@ def listDatabases(self, pattern: Optional[str] = None) -> List[Database]: >>> spark.catalog.listDatabases() [Database(name='default', catalog='spark_catalog', description='default database', ... - >>> spark.catalog.listDatabases("def*") + >>> spark.catalog.listDatabases("def%") [Database(name='default', catalog='spark_catalog', description='default database', ... - >>> spark.catalog.listDatabases("def2*") + >>> spark.catalog.listDatabases("def2%") [] """ if pattern is None: @@ -342,10 +342,10 @@ def listTables( >>> spark.catalog.listTables() [Table(name='test_view', catalog=None, namespace=[], description=None, ... - >>> spark.catalog.listTables(pattern="test*") + >>> spark.catalog.listTables(pattern="test%") [Table(name='test_view', catalog=None, namespace=[], description=None, ... - >>> spark.catalog.listTables(pattern="table*") + >>> spark.catalog.listTables(pattern="table%") [] >>> _ = spark.catalog.dropTempView("test_view") @@ -470,10 +470,10 @@ def listFunctions( >>> spark.catalog.listFunctions() [Function(name=... - >>> spark.catalog.listFunctions(pattern="to_*") + >>> spark.catalog.listFunctions(pattern="to_%") [Function(name=... - >>> spark.catalog.listFunctions(pattern="*not_existing_func*") + >>> spark.catalog.listFunctions(pattern="%not_existing_func%") [] """ if dbName is None: diff --git a/python/pyspark/sql/tests/test_catalog.py b/python/pyspark/sql/tests/test_catalog.py index bc6bfdd2759f2..b890fde3de285 100644 --- a/python/pyspark/sql/tests/test_catalog.py +++ b/python/pyspark/sql/tests/test_catalog.py @@ -42,9 +42,9 @@ def test_list_databases(self): spark.sql("CREATE DATABASE some_db") databases = [db.name for db in spark.catalog.listDatabases()] self.assertEqual(sorted(databases), ["default", "some_db"]) - databases = [db.name for db in spark.catalog.listDatabases("def*")] + databases = [db.name for db in spark.catalog.listDatabases("def%")] self.assertEqual(sorted(databases), ["default"]) - databases = [db.name for db in spark.catalog.listDatabases("def2*")] + databases = [db.name for db in spark.catalog.listDatabases("def2%")] self.assertEqual(sorted(databases), []) def test_database_exists(self): @@ -94,17 +94,17 @@ def test_list_tables(self): tables = sorted(spark.catalog.listTables(), key=lambda t: t.name) tablesWithPattern = sorted( - spark.catalog.listTables(pattern="tab*"), key=lambda t: t.name + spark.catalog.listTables(pattern="tab%"), key=lambda t: t.name ) tablesDefault = sorted( spark.catalog.listTables("default"), key=lambda t: t.name ) tablesDefaultWithPattern = sorted( - spark.catalog.listTables("default", "tab*"), key=lambda t: t.name + spark.catalog.listTables("default", "tab%"), key=lambda t: t.name ) tablesSomeDb = sorted(spark.catalog.listTables("some_db"), key=lambda t: t.name) tablesSomeDbWithPattern = sorted( - spark.catalog.listTables("some_db", "tab*"), key=lambda t: t.name + spark.catalog.listTables("some_db", "tab%"), key=lambda t: t.name ) self.assertEqual(tables, tablesDefault) self.assertEqual(tablesWithPattern, tablesDefaultWithPattern) @@ -265,10 +265,10 @@ def test_list_functions(self): self.assertEqual(functions, functionsDefault) functionsWithPattern = dict( - (f.name, f) for f in spark.catalog.listFunctions(pattern="to*") + (f.name, f) for f in spark.catalog.listFunctions(pattern="to%") ) functionsDefaultWithPattern = dict( - (f.name, f) for f in spark.catalog.listFunctions("default", "to*") + (f.name, f) for f in spark.catalog.listFunctions("default", "to%") ) self.assertTrue(len(functionsWithPattern) > 10) self.assertFalse("+" in functionsWithPattern) @@ -279,7 +279,7 @@ def test_list_functions(self): self.assertTrue("to_unix_timestamp" in functionsWithPattern) self.assertEqual(functionsWithPattern, functionsDefaultWithPattern) functionsWithPattern = dict( - (f.name, f) for f in spark.catalog.listFunctions(pattern="*not_existing_func*") + (f.name, f) for f in spark.catalog.listFunctions(pattern="%not_existing_func%") ) self.assertTrue(len(functionsWithPattern) == 0) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index 74700789dde0f..c2a177c01c8c5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -17,7 +17,16 @@ package org.apache.spark.sql.connector.catalog; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import scala.jdk.javaapi.CollectionConverters; + import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.catalyst.util.StringUtils; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; @@ -26,10 +35,6 @@ import org.apache.spark.sql.errors.QueryExecutionErrors; import org.apache.spark.sql.types.StructType; -import java.util.Collections; -import java.util.Map; -import java.util.Set; - /** * Catalog methods for working with Tables. *
@@ -97,6 +102,30 @@ public interface TableCatalog extends CatalogPlugin { */ Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException; + /** + * List the tables in a namespace from the catalog by pattern string. + *
+ * If the catalog supports views, this must return identifiers for only tables and not views.
+ *
+ * @param namespace a multi-part namespace
+ * @param pattern the filter pattern,
+ * when 'spark.sql.legacy.useVerticalBarAndStarAsWildcardsInLikePattern'
+ * is true, use '*' for any character(s) and '|' for a choice as wildcards.
+ * If it is false, use '%' for any character(s) and '_' for a single character
+ * as wildcards. Please refer to 'regex_pattern' in https://
+ * spark.apache.org/docs/latest/sql-ref-syntax-aux-show-tables.html#parameters
+ * for more details.
+ * @return an array of Identifiers for tables
+ * @throws NoSuchNamespaceException If the namespace does not exist (optional).
+ */
+ default Identifier[] listTables(String[] namespace, String pattern)
+ throws NoSuchNamespaceException {
+ List
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index d17d0a97387d9..6ecc07cbed194 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -1033,7 +1033,8 @@ class SessionCatalog(
* Note that, if the specified database is global temporary view database, we will list global
* temporary views.
*/
- def listTables(db: String): Seq[TableIdentifier] = listTables(db, "*")
+ def listTables(db: String): Seq[TableIdentifier] =
+ listTables(db, StringUtils.getMatchAllWildcard)
/**
* List all matching tables in the specified database, including local temporary views.
@@ -1830,7 +1831,8 @@ class SessionCatalog(
* returns the function identifier and the scope in which it was defined (system or user
* defined).
*/
- def listFunctions(db: String): Seq[(FunctionIdentifier, String)] = listFunctions(db, "*")
+ def listFunctions(db: String): Seq[(FunctionIdentifier, String)] =
+ listFunctions(db, StringUtils.getMatchAllWildcard)
/**
* List all matching functions in the specified database, including temporary functions. This
@@ -1882,7 +1884,7 @@ class SessionCatalog(
dropTable(table, ignoreIfNotExists = false, purge = false)
}
// Temp functions are dropped below, we only need to drop permanent functions here.
- externalCatalog.listFunctions(DEFAULT_DATABASE, "*").map { f =>
+ externalCatalog.listFunctions(DEFAULT_DATABASE, StringUtils.getMatchAllWildcard).map { f =>
FunctionIdentifier(f, Some(DEFAULT_DATABASE))
}.foreach(dropFunction(_, ignoreIfNotExists = false))
clearTempTables()
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
index f94a0650cce41..7e545e102f555 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
@@ -108,19 +108,39 @@ object StringUtils extends Logging {
// scalastyle:on caselocale
/**
- * This utility can be used for filtering pattern in the "Like" of "Show Tables / Functions" DDL
+ * get Wildcard that represent matching "all"
+ */
+ def getMatchAllWildcard: String = {
+ if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) {
+ "*"
+ } else {
+ "%"
+ }
+ }
+
+ def filterPattern(names: Seq[String], pattern: String): Seq[String] = {
+ if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) {
+ legacyFilterPattern(names, pattern)
+ } else {
+ filterBySQLLikePattern(names, pattern)
+ }
+ }
+
+ /**
+ * This legacy utility can be used for filtering pattern in the "Like" of
+ * "Show Tables / Functions" DDL.
* @param names the names list to be filtered
* @param pattern the filter pattern, only '*' and '|' are allowed as wildcards, others will
* follow regular expression convention, case insensitive match and white spaces
* on both ends will be ignored
* @return the filtered names list in order
*/
- def filterPattern(names: Seq[String], pattern: String): Seq[String] = {
+ def legacyFilterPattern(names: Seq[String], pattern: String): Seq[String] = {
val funcNames = scala.collection.mutable.SortedSet.empty[String]
pattern.trim().split("\\|").foreach { subPattern =>
try {
val regex = ("(?i)" + subPattern.replaceAll("\\*", ".*")).r
- funcNames ++= names.filter{ name => regex.pattern.matcher(name).matches() }
+ funcNames ++= names.filter { name => regex.pattern.matcher(name).matches() }
} catch {
case _: PatternSyntaxException =>
}
@@ -128,6 +148,44 @@ object StringUtils extends Logging {
funcNames.toSeq
}
+ /**
+ * This utility can be used for filtering pattern in the "Like" of "Show Tables / Functions" DDL.
+ * @param names the names list to be filtered
+ * @param pattern the filter pattern, same as SQL type `like` expressions:
+ * '%' for any character(s), and '_' for a single character
+ * @return the filtered names list
+ */
+ def filterBySQLLikePattern(names: Seq[String], pattern: String): Seq[String] = {
+ try {
+ val p = Pattern.compile(likePatternToRegExp(pattern), Pattern.CASE_INSENSITIVE)
+ names.filter { name => p.matcher(name).matches() }
+ } catch {
+ case _: PatternSyntaxException => Seq.empty[String]
+ }
+ }
+
+ private[util] def likePatternToRegExp(pattern: String): String = {
+ val regExp = new StringBuilder()
+
+ var index = 0
+ while (index < pattern.length) {
+ val cur = pattern.charAt(index)
+ cur match {
+ // Make a special case for "\\_" and "\\%"
+ case '\\' if (index + 1 < pattern.length()
+ && (pattern.charAt(index + 1) == '_' || pattern.charAt(index + 1) == '%')) =>
+ regExp += pattern.charAt(index + 1)
+ index = index + 1
+ case '_' => regExp ++= "."
+ case '%' => regExp ++= ".*?"
+ case _ => regExp ++= Pattern.quote(Character.toString(cur))
+ }
+ index = index + 1
+ }
+
+ regExp.result()
+ }
+
/**
* A string concatenator for plan strings. Uses length from a configured value, and
* prints a warning the first time a plan is truncated.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 04b392d0c44f4..d5303ee7ce082 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -4604,6 +4604,16 @@ object SQLConf {
.booleanConf
.createWithDefault(false)
+ val LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_PATTERN =
+ buildConf("spark.sql.legacy.useVerticalBarAndStarAsWildcardsInLikePattern")
+ .internal()
+ .doc("When set to true, use '*' for any character(s) and '|' for a choice as wildcards " +
+ "in the like pattern. If it is false, use '%' for any character(s) and '_' for " +
+ "a single character as wildcards in the like pattern.")
+ .version("4.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
// Deprecate "spark.connect.copyFromLocalToFs.allowDestLocal" in favor of this config. This is
// currently optional because we don't want to break existing users who are using the old config.
// If this config is set, then we override the deprecated config.
@@ -5578,6 +5588,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf {
def legacyIgnoreParenthesesAroundStar: Boolean =
getConf(SQLConf.LEGACY_IGNORE_PARENTHESES_AROUND_STAR)
+ def legacyUseStarAndVerticalBarAsWildcardsInLikePattern: Boolean =
+ getConf(SQLConf.LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_PATTERN)
+
/** ********************** SQLConf functionality methods ************ */
/** Set Spark SQL configuration properties. */
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
index ba01f95591613..c20832b4b2b79 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
@@ -92,8 +92,8 @@ abstract class ExternalCatalogSuite extends SparkFunSuite {
test("list databases with pattern") {
val catalog = newBasicCatalog()
assert(catalog.listDatabases("db").toSet == Set.empty)
- assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
- assert(catalog.listDatabases("*1").toSet == Set("db1"))
+ assert(catalog.listDatabases("db%").toSet == Set("db1", "db2", "db3"))
+ assert(catalog.listDatabases("%1").toSet == Set("db1"))
assert(catalog.listDatabases("db2").toSet == Set("db2"))
}
@@ -313,11 +313,11 @@ abstract class ExternalCatalogSuite extends SparkFunSuite {
test("list tables with pattern") {
val catalog = newBasicCatalog()
- intercept[AnalysisException] { catalog.listTables("unknown_db", "*") }
- assert(catalog.listTables("db1", "*").toSet == Set.empty)
- assert(catalog.listTables("db2", "*").toSet == Set("tbl1", "tbl2"))
- assert(catalog.listTables("db2", "tbl*").toSet == Set("tbl1", "tbl2"))
- assert(catalog.listTables("db2", "*1").toSet == Set("tbl1"))
+ intercept[AnalysisException] { catalog.listTables("unknown_db", "%") }
+ assert(catalog.listTables("db1", "%").toSet == Set.empty)
+ assert(catalog.listTables("db2", "%").toSet == Set("tbl1", "tbl2"))
+ assert(catalog.listTables("db2", "tbl%").toSet == Set("tbl1", "tbl2"))
+ assert(catalog.listTables("db2", "%1").toSet == Set("tbl1"))
}
test("column names should be case-preserving and column nullability should be retained") {
@@ -763,7 +763,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite {
val catalog = newEmptyCatalog()
catalog.createDatabase(newDb("mydb"), ignoreIfExists = false)
catalog.createFunction("mydb", newFunc("myfunc"))
- assert(catalog.listFunctions("mydb", "*").toSet == Set("myfunc"))
+ assert(catalog.listFunctions("mydb", "%").toSet == Set("myfunc"))
}
test("create function when database does not exist") {
@@ -782,9 +782,9 @@ abstract class ExternalCatalogSuite extends SparkFunSuite {
test("drop function") {
val catalog = newBasicCatalog()
- assert(catalog.listFunctions("db2", "*").toSet == Set("func1"))
+ assert(catalog.listFunctions("db2", "%").toSet == Set("func1"))
catalog.dropFunction("db2", "func1")
- assert(catalog.listFunctions("db2", "*").isEmpty)
+ assert(catalog.listFunctions("db2", "%").isEmpty)
}
test("drop function when database does not exist") {
@@ -856,8 +856,8 @@ abstract class ExternalCatalogSuite extends SparkFunSuite {
val catalog = newBasicCatalog()
catalog.createFunction("db2", newFunc("func2"))
catalog.createFunction("db2", newFunc("not_me"))
- assert(catalog.listFunctions("db2", "*").toSet == Set("func1", "func2", "not_me"))
- assert(catalog.listFunctions("db2", "func*").toSet == Set("func1", "func2"))
+ assert(catalog.listFunctions("db2", "%").toSet == Set("func1", "func2", "not_me"))
+ assert(catalog.listFunctions("db2", "func%").toSet == Set("func1", "func2"))
}
// --------------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
index e9a60ff17fc1b..51557926a1bd2 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
@@ -244,8 +244,8 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
test("list databases with pattern") {
withBasicCatalog { catalog =>
assert(catalog.listDatabases("db").toSet == Set.empty)
- assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
- assert(catalog.listDatabases("*1").toSet == Set("db1"))
+ assert(catalog.listDatabases("db%").toSet == Set("db1", "db2", "db3"))
+ assert(catalog.listDatabases("%1").toSet == Set("db1"))
assert(catalog.listDatabases("db2").toSet == Set("db2"))
}
}
@@ -867,17 +867,17 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
val tempTable = Range(1, 10, 2, 10)
createTempView(catalog, "tbl1", tempTable, overrideIfExists = false)
createTempView(catalog, "tbl4", tempTable, overrideIfExists = false)
- assert(catalog.listTables("db1", "*").toSet == catalog.listTables("db1").toSet)
- assert(catalog.listTables("db2", "*").toSet == catalog.listTables("db2").toSet)
- assert(catalog.listTables("db2", "tbl*").toSet ==
+ assert(catalog.listTables("db1", "%").toSet == catalog.listTables("db1").toSet)
+ assert(catalog.listTables("db2", "%").toSet == catalog.listTables("db2").toSet)
+ assert(catalog.listTables("db2", "tbl%").toSet ==
Set(TableIdentifier("tbl1"),
TableIdentifier("tbl4"),
TableIdentifier("tbl1", Some("db2")),
TableIdentifier("tbl2", Some("db2"))))
- assert(catalog.listTables("db2", "*1").toSet ==
+ assert(catalog.listTables("db2", "%1").toSet ==
Set(TableIdentifier("tbl1"), TableIdentifier("tbl1", Some("db2"))))
intercept[NoSuchDatabaseException] {
- catalog.listTables("unknown_db", "*")
+ catalog.listTables("unknown_db", "%")
}
}
}
@@ -891,24 +891,24 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
createTempView(catalog, "temp_view1", tempTable, overrideIfExists = false)
createTempView(catalog, "temp_view4", tempTable, overrideIfExists = false)
- assert(catalog.listTables("mydb").toSet == catalog.listTables("mydb", "*").toSet)
- assert(catalog.listTables("mydb").toSet == catalog.listTables("mydb", "*", true).toSet)
+ assert(catalog.listTables("mydb").toSet == catalog.listTables("mydb", "%").toSet)
+ assert(catalog.listTables("mydb").toSet == catalog.listTables("mydb", "%", true).toSet)
assert(catalog.listTables("mydb").toSet ==
- catalog.listTables("mydb", "*", false).toSet ++ catalog.listLocalTempViews("*"))
- assert(catalog.listTables("mydb", "*", true).toSet ==
+ catalog.listTables("mydb", "%", false).toSet ++ catalog.listLocalTempViews("%"))
+ assert(catalog.listTables("mydb", "%", true).toSet ==
Set(TableIdentifier("tbl1", Some("mydb")),
TableIdentifier("tbl2", Some("mydb")),
TableIdentifier("temp_view1"),
TableIdentifier("temp_view4")))
- assert(catalog.listTables("mydb", "*", false).toSet ==
+ assert(catalog.listTables("mydb", "%", false).toSet ==
Set(TableIdentifier("tbl1", Some("mydb")), TableIdentifier("tbl2", Some("mydb"))))
- assert(catalog.listTables("mydb", "tbl*", true).toSet ==
+ assert(catalog.listTables("mydb", "tbl%", true).toSet ==
Set(TableIdentifier("tbl1", Some("mydb")), TableIdentifier("tbl2", Some("mydb"))))
- assert(catalog.listTables("mydb", "tbl*", false).toSet ==
+ assert(catalog.listTables("mydb", "tbl%", false).toSet ==
Set(TableIdentifier("tbl1", Some("mydb")), TableIdentifier("tbl2", Some("mydb"))))
- assert(catalog.listTables("mydb", "temp_view*", true).toSet ==
+ assert(catalog.listTables("mydb", "temp_view%", true).toSet ==
Set(TableIdentifier("temp_view1"), TableIdentifier("temp_view4")))
- assert(catalog.listTables("mydb", "temp_view*", false).toSet == Set.empty)
+ assert(catalog.listTables("mydb", "temp_view%", false).toSet == Set.empty)
}
}
@@ -917,11 +917,11 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
val tempTable = Range(1, 10, 2, 10)
createTempView(catalog, "temp_view1", tempTable, overrideIfExists = false)
createTempView(catalog, "temp_view4", tempTable, overrideIfExists = false)
- assert(catalog.listLocalTempViews("*").toSet ==
+ assert(catalog.listLocalTempViews("%").toSet ==
Set(TableIdentifier("temp_view1"), TableIdentifier("temp_view4")))
- assert(catalog.listLocalTempViews("temp_view*").toSet ==
+ assert(catalog.listLocalTempViews("temp_view%").toSet ==
Set(TableIdentifier("temp_view1"), TableIdentifier("temp_view4")))
- assert(catalog.listLocalTempViews("*1").toSet == Set(TableIdentifier("temp_view1")))
+ assert(catalog.listLocalTempViews("%1").toSet == Set(TableIdentifier("temp_view1")))
assert(catalog.listLocalTempViews("does_not_exist").toSet == Set.empty)
}
}
@@ -933,15 +933,15 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
createTempView(catalog, "temp_view4", tempTable, overrideIfExists = false)
createGlobalTempView(catalog, "global_temp_view1", tempTable, overrideIfExists = false)
createGlobalTempView(catalog, "global_temp_view2", tempTable, overrideIfExists = false)
- assert(catalog.listTables(catalog.globalTempViewManager.database, "*").toSet ==
+ assert(catalog.listTables(catalog.globalTempViewManager.database, "%").toSet ==
Set(TableIdentifier("temp_view1"),
TableIdentifier("temp_view4"),
TableIdentifier("global_temp_view1", Some(catalog.globalTempViewManager.database)),
TableIdentifier("global_temp_view2", Some(catalog.globalTempViewManager.database))))
- assert(catalog.listTables(catalog.globalTempViewManager.database, "*temp_view1").toSet ==
+ assert(catalog.listTables(catalog.globalTempViewManager.database, "%temp_view1").toSet ==
Set(TableIdentifier("temp_view1"),
TableIdentifier("global_temp_view1", Some(catalog.globalTempViewManager.database))))
- assert(catalog.listTables(catalog.globalTempViewManager.database, "global*").toSet ==
+ assert(catalog.listTables(catalog.globalTempViewManager.database, "global%").toSet ==
Set(TableIdentifier("global_temp_view1", Some(catalog.globalTempViewManager.database)),
TableIdentifier("global_temp_view2", Some(catalog.globalTempViewManager.database))))
}
@@ -1534,11 +1534,11 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
withEmptyCatalog { catalog =>
catalog.createDatabase(newDb("mydb"), ignoreIfExists = false)
catalog.createFunction(newFunc("myfunc", Some("mydb")), ignoreIfExists = false)
- assert(catalog.externalCatalog.listFunctions("mydb", "*").toSet == Set("myfunc"))
+ assert(catalog.externalCatalog.listFunctions("mydb", "%").toSet == Set("myfunc"))
// Create function without explicitly specifying database
catalog.setCurrentDatabase("mydb")
catalog.createFunction(newFunc("myfunc2"), ignoreIfExists = false)
- assert(catalog.externalCatalog.listFunctions("mydb", "*").toSet == Set("myfunc", "myfunc2"))
+ assert(catalog.externalCatalog.listFunctions("mydb", "%").toSet == Set("myfunc", "myfunc2"))
}
}
@@ -1621,7 +1621,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
assert(catalog.isTemporaryFunction(FunctionIdentifier("temp1")))
// Returns false when the function is permanent
- assert(catalog.externalCatalog.listFunctions("db2", "*").toSet == Set("func1"))
+ assert(catalog.externalCatalog.listFunctions("db2", "%").toSet == Set("func1"))
assert(!catalog.isTemporaryFunction(FunctionIdentifier("func1", Some("db2"))))
assert(!catalog.isTemporaryFunction(FunctionIdentifier("db2.func1")))
catalog.setCurrentDatabase("db2")
@@ -1672,16 +1672,16 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
test("drop function") {
withBasicCatalog { catalog =>
- assert(catalog.externalCatalog.listFunctions("db2", "*").toSet == Set("func1"))
+ assert(catalog.externalCatalog.listFunctions("db2", "%").toSet == Set("func1"))
catalog.dropFunction(
FunctionIdentifier("func1", Some("db2")), ignoreIfNotExists = false)
- assert(catalog.externalCatalog.listFunctions("db2", "*").isEmpty)
+ assert(catalog.externalCatalog.listFunctions("db2", "%").isEmpty)
// Drop function without explicitly specifying database
catalog.setCurrentDatabase("db2")
catalog.createFunction(newFunc("func2", Some("db2")), ignoreIfExists = false)
- assert(catalog.externalCatalog.listFunctions("db2", "*").toSet == Set("func2"))
+ assert(catalog.externalCatalog.listFunctions("db2", "%").toSet == Set("func2"))
catalog.dropFunction(FunctionIdentifier("func2"), ignoreIfNotExists = false)
- assert(catalog.externalCatalog.listFunctions("db2", "*").isEmpty)
+ assert(catalog.externalCatalog.listFunctions("db2", "%").isEmpty)
}
}
@@ -1781,16 +1781,16 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
funcMeta1, overrideIfExists = false, functionBuilder = Some(tempFunc1))
catalog.registerFunction(
funcMeta2, overrideIfExists = false, functionBuilder = Some(tempFunc2))
- assert(catalog.listFunctions("db1", "*").map(_._1).toSet ==
+ assert(catalog.listFunctions("db1", "%").map(_._1).toSet ==
Set(FunctionIdentifier("func1"),
FunctionIdentifier("yes_me")))
- assert(catalog.listFunctions("db2", "*").map(_._1).toSet ==
+ assert(catalog.listFunctions("db2", "%").map(_._1).toSet ==
Set(FunctionIdentifier("func1"),
FunctionIdentifier("yes_me"),
FunctionIdentifier("func1", Some("db2"), Some(SESSION_CATALOG_NAME)),
FunctionIdentifier("func2", Some("db2"), Some(SESSION_CATALOG_NAME)),
FunctionIdentifier("not_me", Some("db2"), Some(SESSION_CATALOG_NAME))))
- assert(catalog.listFunctions("db2", "func*").map(_._1).toSet ==
+ assert(catalog.listFunctions("db2", "func%").map(_._1).toSet ==
Set(FunctionIdentifier("func1"),
FunctionIdentifier("func1", Some("db2"), Some(SESSION_CATALOG_NAME)),
FunctionIdentifier("func2", Some("db2"), Some(SESSION_CATALOG_NAME))))
@@ -1818,7 +1818,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
// Load func2 into the function registry.
catalog.registerFunction(func2, overrideIfExists = false, functionBuilder = Some(builder))
// Should not include func2.
- assert(catalog.listFunctions("default", "*").map(_._1).toSet ==
+ assert(catalog.listFunctions("default", "%").map(_._1).toSet ==
Set(FunctionIdentifier("func1"),
FunctionIdentifier("func1", Some("default"), Some(SESSION_CATALOG_NAME)))
)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index b306ca3cd18a5..3749afd62225a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -2175,11 +2175,11 @@ class DDLParserSuite extends AnalysisTest {
parsePlan("SHOW VIEWS"),
ShowViews(CurrentNamespace, None))
comparePlans(
- parsePlan("SHOW VIEWS '*test*'"),
- ShowViews(CurrentNamespace, Some("*test*")))
+ parsePlan("SHOW VIEWS '%test%'"),
+ ShowViews(CurrentNamespace, Some("%test%")))
comparePlans(
- parsePlan("SHOW VIEWS LIKE '*test*'"),
- ShowViews(CurrentNamespace, Some("*test*")))
+ parsePlan("SHOW VIEWS LIKE '%test%'"),
+ ShowViews(CurrentNamespace, Some("%test%")))
comparePlans(
parsePlan("SHOW VIEWS FROM testcat.ns1.ns2.tbl"),
ShowViews(UnresolvedNamespace(Seq("testcat", "ns1", "ns2", "tbl")), None))
@@ -2187,11 +2187,11 @@ class DDLParserSuite extends AnalysisTest {
parsePlan("SHOW VIEWS IN testcat.ns1.ns2.tbl"),
ShowViews(UnresolvedNamespace(Seq("testcat", "ns1", "ns2", "tbl")), None))
comparePlans(
- parsePlan("SHOW VIEWS IN ns1 '*test*'"),
- ShowViews(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+ parsePlan("SHOW VIEWS IN ns1 '%test%'"),
+ ShowViews(UnresolvedNamespace(Seq("ns1")), Some("%test%")))
comparePlans(
- parsePlan("SHOW VIEWS IN ns1 LIKE '*test*'"),
- ShowViews(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+ parsePlan("SHOW VIEWS IN ns1 LIKE '%test%'"),
+ ShowViews(UnresolvedNamespace(Seq("ns1")), Some("%test%")))
}
test("analyze table statistics") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala
index e3a46d276923a..a11f4b8007375 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala
@@ -55,16 +55,16 @@ class StringUtilsSuite extends SparkFunSuite with SQLHelper {
assert(escapeLikeRegex("a_b", '\"') === expectedEscapedStrSeven)
}
- test("filter pattern") {
+ test("legacy filter pattern") {
val names = Seq("a1", "a2", "b2", "c3")
- assert(filterPattern(names, " * ") === Seq("a1", "a2", "b2", "c3"))
- assert(filterPattern(names, "*a*") === Seq("a1", "a2"))
- assert(filterPattern(names, " *a* ") === Seq("a1", "a2"))
- assert(filterPattern(names, " a* ") === Seq("a1", "a2"))
- assert(filterPattern(names, " a.* ") === Seq("a1", "a2"))
- assert(filterPattern(names, " B.*|a* ") === Seq("a1", "a2", "b2"))
- assert(filterPattern(names, " a. ") === Seq("a1", "a2"))
- assert(filterPattern(names, " d* ") === Nil)
+ assert(legacyFilterPattern(names, " * ") === Seq("a1", "a2", "b2", "c3"))
+ assert(legacyFilterPattern(names, "*a*") === Seq("a1", "a2"))
+ assert(legacyFilterPattern(names, " *a* ") === Seq("a1", "a2"))
+ assert(legacyFilterPattern(names, " a* ") === Seq("a1", "a2"))
+ assert(legacyFilterPattern(names, " a.* ") === Seq("a1", "a2"))
+ assert(legacyFilterPattern(names, " B.*|a* ") === Seq("a1", "a2", "b2"))
+ assert(legacyFilterPattern(names, " a. ") === Seq("a1", "a2"))
+ assert(legacyFilterPattern(names, " d* ") === Nil)
}
test("string concatenation") {
@@ -146,4 +146,38 @@ class StringUtilsSuite extends SparkFunSuite with SQLHelper {
val expectedOutput = Seq("`c1`", "`v2.c2`", "`v1`.`c2`")
assert(orderSuggestedIdentifiersBySimilarity(baseString, testStrings) === expectedOutput)
}
+
+ test("like pattern to regular expression") {
+ assert(likePatternToRegExp("\\_") === "_")
+ assert(likePatternToRegExp("\\%") === "%")
+
+ assert(likePatternToRegExp("_") === ".")
+ assert(likePatternToRegExp("%") === ".*?")
+
+ assert(likePatternToRegExp(" * ") === "\\Q \\E\\Q*\\E\\Q \\E")
+ assert(likePatternToRegExp("*a*") === "\\Q*\\E\\Qa\\E\\Q*\\E")
+ assert(likePatternToRegExp(" *a* ") === "\\Q \\E\\Q*\\E\\Qa\\E\\Q*\\E\\Q \\E")
+ assert(likePatternToRegExp(" a* ") === "\\Q \\E\\Qa\\E\\Q*\\E\\Q \\E")
+ assert(likePatternToRegExp(" a.* ") === "\\Q \\E\\Qa\\E\\Q.\\E\\Q*\\E\\Q \\E")
+ assert(likePatternToRegExp(" B.*|a* ") ===
+ "\\Q \\E\\QB\\E\\Q.\\E\\Q*\\E\\Q|\\E\\Qa\\E\\Q*\\E\\Q \\E")
+ assert(likePatternToRegExp(" a. ") === "\\Q \\E\\Qa\\E\\Q.\\E\\Q \\E")
+ assert(likePatternToRegExp(" d* ") === "\\Q \\E\\Qd\\E\\Q*\\E\\Q \\E")
+ }
+
+ test("filterBySQLLikePattern") {
+ val names = Seq("a1", "a2", "b2", "c3")
+ assert(filterBySQLLikePattern(names, "%") === Seq("a1", "a2", "b2", "c3"))
+ assert(filterBySQLLikePattern(names, "%a%") === Seq("a1", "a2"))
+ assert(filterBySQLLikePattern(names, "a%") === Seq("a1", "a2"))
+ assert(filterBySQLLikePattern(names, "%a") === Nil)
+ assert(filterBySQLLikePattern(names, "a_") === Seq("a1", "a2"))
+ assert(filterBySQLLikePattern(names, "a.*") === Nil)
+ assert(filterBySQLLikePattern(names, "B%|a%") === Nil)
+ assert(filterBySQLLikePattern(names, "A%") === Seq("a1", "a2"))
+ assert(filterBySQLLikePattern(names, "B%") === Seq("b2"))
+ assert(filterBySQLLikePattern(names, "_2") === Seq("a2", "b2"))
+ assert(filterBySQLLikePattern(names, "d%") === Nil)
+ assert(filterBySQLLikePattern(names, "\\s") === Nil)
+ }
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala
index 2935b01649cad..fe7ce139111af 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala
@@ -24,6 +24,7 @@ import scala.jdk.CollectionConverters._
import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUnsupportedOperationException}
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException}
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.util.quoteIdentifier
@@ -33,7 +34,7 @@ import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
-class CatalogSuite extends SparkFunSuite {
+class CatalogSuite extends SparkFunSuite with SQLConfHelper {
import CatalogV2Implicits._
private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String]
@@ -96,6 +97,43 @@ class CatalogSuite extends SparkFunSuite {
assert(catalog.listTables(Array("ns2")).toSet == Set(ident3))
}
+ test("listTables with pattern") {
+ val catalog = newCatalog()
+ val ident1 = Identifier.of(Array("ns"), "test_table_1")
+ val ident2 = Identifier.of(Array("ns"), "test_table_2")
+ val ident3 = Identifier.of(Array("ns2"), "test_table_1")
+
+ Seq(("true", "*test*"), ("false", "%test%")).foreach { v =>
+ withSQLConf(
+ SQLConf.LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_PATTERN.key -> v._1) {
+ intercept[NoSuchNamespaceException](catalog.listTables(Array("ns", v._2)))
+
+ catalog.createTable(ident1, schema, emptyTrans, emptyProps)
+
+ assert(catalog.listTables(Array("ns"), v._2).toSet == Set(ident1))
+ intercept[NoSuchNamespaceException](catalog.listTables(Array("ns2"), v._2))
+
+ catalog.createTable(ident3, schema, emptyTrans, emptyProps)
+ catalog.createTable(ident2, schema, emptyTrans, emptyProps)
+
+ assert(catalog.listTables(Array("ns"), v._2).toSet == Set(ident1, ident2))
+ assert(catalog.listTables(Array("ns2"), v._2).toSet == Set(ident3))
+
+ catalog.dropTable(ident1)
+
+ assert(catalog.listTables(Array("ns"), v._2).toSet == Set(ident2))
+
+ catalog.dropTable(ident2)
+
+ assert(catalog.listTables(Array("ns"), v._2).isEmpty)
+ assert(catalog.listTables(Array("ns2"), v._2).toSet == Set(ident3))
+
+ catalog.dropNamespace(Array("ns"), cascade = true)
+ catalog.dropNamespace(Array("ns2"), cascade = true)
+ }
+ }
+ }
+
test("createTable") {
val catalog = newCatalog()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
index b9a7151b4aff4..7a4aa22594446 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
@@ -175,7 +175,7 @@ case class ShowFunctionsCommand(
// match any sequence of characters (including no characters).
val functionNames =
sparkSession.sessionState.catalog
- .listFunctions(db, pattern.getOrElse("*"))
+ .listFunctions(db, pattern.getOrElse(StringUtils.getMatchAllWildcard))
.collect {
case (f, "USER") if showUserFunctions => f.unquotedString
case (f, "SYSTEM") if showSystemFunctions => f.unquotedString
@@ -186,8 +186,8 @@ case class ShowFunctionsCommand(
// only show when showSystemFunctions=true
if (showSystemFunctions) {
(functionNames ++
- StringUtils.filterPattern(
- FunctionRegistry.builtinOperators.keys.toSeq, pattern.getOrElse("*")))
+ StringUtils.filterPattern(FunctionRegistry.builtinOperators.keys.toSeq,
+ pattern.getOrElse(StringUtils.getMatchAllWildcard)))
.sorted.map(Row(_))
} else {
functionNames.sorted.map(Row(_))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
index d71d0d43683cb..052408cc912c2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, GlobalTempView,
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, TemporaryViewRelation}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, SubqueryExpression, VariableReference}
import org.apache.spark.sql.catalyst.plans.logical.{AnalysisOnlyCommand, CTEInChildren, CTERelationDef, LogicalPlan, Project, View, WithCTE}
-import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
@@ -336,7 +336,7 @@ case class ShowViewsCommand(
// Show the information of views.
val views = tableIdentifierPattern.map(catalog.listViews(databaseName, _))
- .getOrElse(catalog.listViews(databaseName, "*"))
+ .getOrElse(catalog.listViews(databaseName, StringUtils.getMatchAllWildcard))
views.map { tableIdent =>
val namespace = tableIdent.database.toArray.quoted
val tableName = tableIdent.table
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowFunctionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowFunctionsExec.scala
index b80f4ee2357d5..f3ed4c6e92191 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowFunctionsExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowFunctionsExec.scala
@@ -40,7 +40,7 @@ case class ShowFunctionsExec(
pattern: Option[String]) extends V2CommandExec with LeafExecNode {
private def applyPattern(names: Seq[String]): Seq[String] = {
- StringUtils.filterPattern(names, pattern.getOrElse("*"))
+ StringUtils.filterPattern(names, pattern.getOrElse(StringUtils.getMatchAllWildcard))
}
override protected def run(): Seq[InternalRow] = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala
index c55c7b9f98544..a4a89aeb5c0b4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala
@@ -53,7 +53,7 @@ case class ShowNamespacesExec(
val rows = new ArrayBuffer[InternalRow]()
namespaceNames.map { ns =>
- if (pattern.map(StringUtils.filterPattern(Seq(ns), _).nonEmpty).getOrElse(true)) {
+ if (pattern.forall(StringUtils.filterPattern(Seq(ns), _).nonEmpty)) {
rows += toCatalystRow(ns)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala
index cde3dfb7ead58..ebfbefdecfa8d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala
@@ -39,7 +39,7 @@ case class ShowTablesExec(
val tables = catalog.listTables(namespace.toArray)
tables.map { table =>
- if (pattern.map(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty).getOrElse(true)) {
+ if (pattern.forall(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty)) {
rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table))
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala
index 0b2d11a597d5c..1e35d33d36fea 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTableType
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName
import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, ToPrettyString}
-import org.apache.spark.sql.catalyst.util.{quoteIdentifier, StringUtils}
+import org.apache.spark.sql.catalyst.util.quoteIdentifier
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, SupportsPartitionManagement, Table, TableCatalog}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
import org.apache.spark.sql.errors.QueryCompilationErrors
@@ -45,16 +45,13 @@ case class ShowTablesExtendedExec(
val rows = new ArrayBuffer[InternalRow]()
// fetch tables
- // TODO We need a new listTable overload that takes a pattern string.
- val tables = catalog.listTables(namespace.toArray)
+ val tables = catalog.listTables(namespace.toArray, pattern)
tables.map { tableIdent =>
- if (StringUtils.filterPattern(Seq(tableIdent.name()), pattern).nonEmpty) {
- val table = catalog.loadTable(tableIdent)
- val information = getTableDetails(catalog.name, tableIdent, table)
- rows += toCatalystRow(tableIdent.namespace().quoted, tableIdent.name(), false,
- s"$information\n")
- }
- }
+ val table = catalog.loadTable(tableIdent)
+ val information = getTableDetails(catalog.name, tableIdent, table)
+ rows += toCatalystRow(tableIdent.namespace().quoted, tableIdent.name(), false,
+ s"$information\n")
+ }
// fetch temp views, includes: global temp view, local temp view
val sessionCatalog = session.sessionState.catalog
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out
index ce5f7995f5d13..f06037ec129d2 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out
@@ -60,37 +60,37 @@ ShowTables [namespace#x, tableName#x, isTemporary#x]
-- !query
-SHOW TABLES 'show_t*'
+SHOW TABLES 'show_t%'
-- !query analysis
-ShowTables show_t*, [namespace#x, tableName#x, isTemporary#x]
+ShowTables show_t%, [namespace#x, tableName#x, isTemporary#x]
+- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb]
-- !query
-SHOW TABLES LIKE 'show_t1*|show_t2*'
+SHOW TABLES LIKE 'show_t%'
-- !query analysis
-ShowTables show_t1*|show_t2*, [namespace#x, tableName#x, isTemporary#x]
+ShowTables show_t%, [namespace#x, tableName#x, isTemporary#x]
+- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb]
-- !query
-SHOW TABLES IN showdb 'show_t*'
+SHOW TABLES IN showdb 'show_t%'
-- !query analysis
-ShowTables show_t*, [namespace#x, tableName#x, isTemporary#x]
+ShowTables show_t%, [namespace#x, tableName#x, isTemporary#x]
+- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb]
-- !query
-SHOW TABLES IN showdb LIKE 'show_t*'
+SHOW TABLES IN showdb LIKE 'show_t%'
-- !query analysis
-ShowTables show_t*, [namespace#x, tableName#x, isTemporary#x]
+ShowTables show_t%, [namespace#x, tableName#x, isTemporary#x]
+- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb]
-- !query
-SHOW TABLE EXTENDED LIKE 'show_t*'
+SHOW TABLE EXTENDED LIKE 'show_t%'
-- !query analysis
-ShowTablesCommand showdb, show_t*, [namespace#x, tableName#x, isTemporary#x, information#x], true
+ShowTablesCommand showdb, show_t%, [namespace#x, tableName#x, isTemporary#x, information#x], true
-- !query
@@ -128,21 +128,21 @@ org.apache.spark.sql.catalyst.parser.ParseException
-- !query
-SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1)
+SHOW TABLE EXTENDED LIKE 'show_t%' PARTITION(c='Us', d=1)
-- !query analysis
org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
"errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
"sqlState" : "42P01",
"messageParameters" : {
- "relationName" : "`show_t*`"
+ "relationName" : "`show_t%`"
},
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
"startIndex" : 26,
"stopIndex" : 34,
- "fragment" : "'show_t*'"
+ "fragment" : "'show_t%'"
} ]
}
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out
index c8f2c6f9cc029..fa3dc2522e08e 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out
@@ -77,31 +77,31 @@ ShowViewsCommand global_temp, [namespace#x, viewName#x, isTemporary#x]
-- !query
-SHOW VIEWS 'view_*'
+SHOW VIEWS 'view_%'
-- !query analysis
-ShowViewsCommand showdb, view_*, [namespace#x, viewName#x, isTemporary#x]
+ShowViewsCommand showdb, view_%, [namespace#x, viewName#x, isTemporary#x]
-- !query
-SHOW VIEWS LIKE 'view_1*|view_2*'
+SHOW VIEWS LIKE 'view_%'
-- !query analysis
-ShowViewsCommand showdb, view_1*|view_2*, [namespace#x, viewName#x, isTemporary#x]
+ShowViewsCommand showdb, view_%, [namespace#x, viewName#x, isTemporary#x]
-- !query
-SHOW VIEWS IN showdb 'view_*'
+SHOW VIEWS IN showdb 'view_%'
-- !query analysis
-ShowViewsCommand showdb, view_*, [namespace#x, viewName#x, isTemporary#x]
+ShowViewsCommand showdb, view_%, [namespace#x, viewName#x, isTemporary#x]
-- !query
-SHOW VIEWS IN showdb LIKE 'view_*'
+SHOW VIEWS IN showdb LIKE 'view_%'
-- !query analysis
-ShowViewsCommand showdb, view_*, [namespace#x, viewName#x, isTemporary#x]
+ShowViewsCommand showdb, view_%, [namespace#x, viewName#x, isTemporary#x]
-- !query
-SHOW VIEWS IN wrongdb LIKE 'view_*'
+SHOW VIEWS IN wrongdb LIKE 'view_%'
-- !query analysis
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
{
diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql b/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql
index 8f46c93ee3233..658d681e4cf9c 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql
@@ -12,13 +12,13 @@ SHOW TABLES;
SHOW TABLES IN showdb;
-- SHOW TABLES WITH wildcard match
-SHOW TABLES 'show_t*';
-SHOW TABLES LIKE 'show_t1*|show_t2*';
-SHOW TABLES IN showdb 'show_t*';
-SHOW TABLES IN showdb LIKE 'show_t*';
+SHOW TABLES 'show_t%';
+SHOW TABLES LIKE 'show_t%';
+SHOW TABLES IN showdb 'show_t%';
+SHOW TABLES IN showdb LIKE 'show_t%';
-- SHOW TABLE EXTENDED
-SHOW TABLE EXTENDED LIKE 'show_t*';
+SHOW TABLE EXTENDED LIKE 'show_t%';
SHOW TABLE EXTENDED;
-- SHOW TABLE EXTENDED ... PARTITION
@@ -26,7 +26,7 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1);
-- Throw a ParseException if table name is not specified.
SHOW TABLE EXTENDED PARTITION(c='Us', d=1);
-- Don't support regular expression for table name if a partition specification is present.
-SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1);
+SHOW TABLE EXTENDED LIKE 'show_t%' PARTITION(c='Us', d=1);
-- Partition specification is not complete.
SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us');
-- Partition specification is invalid.
diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-views.sql b/sql/core/src/test/resources/sql-tests/inputs/show-views.sql
index bdf9ef4aa6780..914375a48eee5 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/show-views.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/show-views.sql
@@ -14,12 +14,12 @@ SHOW VIEWS IN showdb;
SHOW VIEWS IN global_temp;
-- SHOW VIEWS WITH wildcard match
-SHOW VIEWS 'view_*';
-SHOW VIEWS LIKE 'view_1*|view_2*';
-SHOW VIEWS IN showdb 'view_*';
-SHOW VIEWS IN showdb LIKE 'view_*';
+SHOW VIEWS 'view_%';
+SHOW VIEWS LIKE 'view_%';
+SHOW VIEWS IN showdb 'view_%';
+SHOW VIEWS IN showdb LIKE 'view_%';
-- Error when database not exists
-SHOW VIEWS IN wrongdb LIKE 'view_*';
+SHOW VIEWS IN wrongdb LIKE 'view_%';
-- Clean Up
DROP VIEW global_temp.view_3;
diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
index 442f0fe5d5f27..207eb631811f4 100644
--- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
@@ -76,7 +76,7 @@ show_t3
-- !query
-SHOW TABLES 'show_t*'
+SHOW TABLES 'show_t%'
-- !query schema
struct