From e418b3d9e0f48770a655ad960a6784ef33220220 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 10 Nov 2023 17:03:59 +0800 Subject: [PATCH 01/11] [SPARK-45880][SQL] Introduce a new TableCatalog.listTable overload that takes a pattern string for v2 catalog --- .../sql/connector/catalog/TableCatalog.java | 33 ++++++++++++++++--- .../sql/connector/catalog/CatalogSuite.scala | 29 ++++++++++++++++ 2 files changed, 58 insertions(+), 4 deletions(-) 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 6642adc33548d..0641412ccecd9 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,26 @@ 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, 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 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 tableNames = Arrays.stream(listTables(namespace)).map(Identifier::name).toList(); + return CollectionConverters.asJava(StringUtils.filterPattern( + CollectionConverters.asScala(tableNames).toSeq(), pattern)).stream().map( + name -> Identifier.of(namespace, name)).toArray(Identifier[]::new); + } + /** * Load table metadata by {@link Identifier identifier} from the catalog. *

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 e79fff7479b9e..9b1f554a9a13d 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 @@ -96,6 +96,35 @@ class CatalogSuite extends SparkFunSuite { assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) } + test("listTables by 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") + + intercept[NoSuchNamespaceException](catalog.listTables(Array("ns", "*test*"))) + + catalog.createTable(ident1, schema, emptyTrans, emptyProps) + + assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident1)) + intercept[NoSuchNamespaceException](catalog.listTables(Array("ns2"))) + + catalog.createTable(ident3, schema, emptyTrans, emptyProps) + catalog.createTable(ident2, schema, emptyTrans, emptyProps) + + assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident1, ident2)) + assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + + catalog.dropTable(ident1) + + assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident2)) + + catalog.dropTable(ident2) + + assert(catalog.listTables(Array("ns"), "*test*").isEmpty) + assert(catalog.listTables(Array("ns2"), "*test*").toSet == Set(ident3)) + } + test("createTable") { val catalog = newCatalog() From 43bfe0fd32f34e6a15111237cf85b2234e5aaf20 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 10 Nov 2023 17:18:58 +0800 Subject: [PATCH 02/11] fix UT --- .../org/apache/spark/sql/connector/catalog/CatalogSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 9b1f554a9a13d..1989dacd579f9 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 @@ -107,13 +107,13 @@ class CatalogSuite extends SparkFunSuite { catalog.createTable(ident1, schema, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident1)) - intercept[NoSuchNamespaceException](catalog.listTables(Array("ns2"))) + intercept[NoSuchNamespaceException](catalog.listTables(Array("ns2"), "*test*")) catalog.createTable(ident3, schema, emptyTrans, emptyProps) catalog.createTable(ident2, schema, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident1, ident2)) - assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + assert(catalog.listTables(Array("ns2"), "*test*").toSet == Set(ident3)) catalog.dropTable(ident1) From c51043f074795df1fa580c04c6b9899078aa71ee Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 19 Feb 2024 20:14:53 +0800 Subject: [PATCH 03/11] update ShowTablesExtendedExec & ShowTablesExec --- .../datasources/v2/ShowTablesExec.scala | 10 +++++----- .../datasources/v2/ShowTablesExtendedExec.scala | 17 +++++++---------- 2 files changed, 12 insertions(+), 15 deletions(-) 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..95af3005f9c41 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 @@ -21,7 +21,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.execution.LeafExecNode @@ -37,11 +36,12 @@ case class ShowTablesExec( override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - val tables = catalog.listTables(namespace.toArray) + val tables = pattern match { + case Some(v) => catalog.listTables(namespace.toArray, v) + case _ => catalog.listTables(namespace.toArray) + } tables.map { table => - if (pattern.map(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty).getOrElse(true)) { - rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table)) - } + rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table)) } rows.toSeq 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 From 55fa734e4a0deebfba5a495fda06920a2ccf2e93 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 20 Feb 2024 09:31:52 +0800 Subject: [PATCH 04/11] update ref --- .../spark/sql/connector/catalog/TableCatalog.java | 3 ++- .../sql/execution/datasources/v2/ShowTablesExec.scala | 10 +++++----- 2 files changed, 7 insertions(+), 6 deletions(-) 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 fa49959e51dcd..21b529d80cae2 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 @@ -110,7 +110,8 @@ public interface TableCatalog extends CatalogPlugin { * @param namespace a multi-part namespace * @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 + * on both ends will be ignored. 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). */ 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 95af3005f9c41..cde3dfb7ead58 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 @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.execution.LeafExecNode @@ -36,12 +37,11 @@ case class ShowTablesExec( override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - val tables = pattern match { - case Some(v) => catalog.listTables(namespace.toArray, v) - case _ => catalog.listTables(namespace.toArray) - } + val tables = catalog.listTables(namespace.toArray) tables.map { table => - rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table)) + if (pattern.map(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty).getOrElse(true)) { + rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table)) + } } rows.toSeq From 3fa8e9a7385e9384c7bc2935c30aa8f54ba57e11 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 20 Feb 2024 09:43:03 +0800 Subject: [PATCH 05/11] update TableCatalog ref --- .../org/apache/spark/sql/connector/catalog/TableCatalog.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 21b529d80cae2..f02ab109fe70c 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 @@ -110,7 +110,8 @@ public interface TableCatalog extends CatalogPlugin { * @param namespace a multi-part namespace * @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. Please refer to regex_pattern in https://spark.apache.org/docs/latest/sql-ref-syntax-aux-show-tables.html#parameters + * on both ends will be ignored. 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). From 596de52124e19b6be0ffc916d0a1e8ce8696b6bc Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 26 Feb 2024 11:20:07 +0800 Subject: [PATCH 06/11] [SPARK-45880][SQL] `Like` pattern accept only SQL type like expressions containing '%' for any character(s), and '_' for a single character --- .../sql/catalyst/catalog/SessionCatalog.scala | 8 +- .../spark/sql/catalyst/util/StringUtils.scala | 57 +++++++++++- .../apache/spark/sql/internal/SQLConf.scala | 13 +++ .../sql/catalyst/parser/DDLParserSuite.scala | 16 ++-- .../sql/catalyst/util/StringUtilsSuite.scala | 50 +++++++++-- .../sql/connector/catalog/CatalogSuite.scala | 39 +++++---- .../sql/execution/command/functions.scala | 6 +- .../spark/sql/execution/command/views.scala | 4 +- .../datasources/v2/ShowFunctionsExec.scala | 2 +- .../datasources/v2/ShowNamespacesExec.scala | 2 +- .../datasources/v2/ShowTablesExec.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 4 +- .../command/ShowFunctionsSuiteBase.scala | 26 +++--- .../command/ShowNamespacesSuiteBase.scala | 8 +- .../command/ShowTablesSuiteBase.scala | 20 ++--- .../command/v1/ShowTablesSuite.scala | 7 +- .../spark/sql/hive/HiveExternalCatalog.scala | 4 +- .../spark/sql/hive/client/HiveShim.scala | 33 +++++-- .../spark/sql/hive/StatisticsSuite.scala | 4 +- .../sql/hive/execution/HiveCommandSuite.scala | 87 ++++++++++--------- .../execution/command/ShowTablesSuite.scala | 2 +- 21 files changed, 267 insertions(+), 127 deletions(-) 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..e1dbd9dec35a0 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.getAllMatchWildcard) /** * 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.getAllMatchWildcard) /** * 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.getAllMatchWildcard).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..bcdf19b78fc98 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 @@ -107,6 +107,22 @@ object StringUtils extends Logging { def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s.trimAll().toLowerCase) // scalastyle:on caselocale + def getAllMatchWildcard: String = { + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + "*" + } else { + "%" + } + } + + def filterPattern(names: Seq[String], pattern: String): Seq[String] = { + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + filterPatternLegacy(names, pattern) + } else { + filterBySQLLikePattern(names, pattern) + } + } + /** * This utility can be used for filtering pattern in the "Like" of "Show Tables / Functions" DDL * @param names the names list to be filtered @@ -115,12 +131,12 @@ object StringUtils extends Logging { * on both ends will be ignored * @return the filtered names list in order */ - def filterPattern(names: Seq[String], pattern: String): Seq[String] = { + def filterPatternLegacy(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 +144,43 @@ object StringUtils extends Logging { funcNames.toSeq } + 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] + } + } + + /** + * + * @param pattern the filter pattern, only '%' and '_' are allowed as wildcards, others will + * follow regular expression convention, + * @return the filtered names list + */ + 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/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 7704469b2986e..0643d093d87cb 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..9a38f86f86967 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 @@ -57,14 +57,14 @@ class StringUtilsSuite extends SparkFunSuite with SQLHelper { test("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(filterPatternLegacy(names, " * ") === Seq("a1", "a2", "b2", "c3")) + assert(filterPatternLegacy(names, "*a*") === Seq("a1", "a2")) + assert(filterPatternLegacy(names, " *a* ") === Seq("a1", "a2")) + assert(filterPatternLegacy(names, " a* ") === Seq("a1", "a2")) + assert(filterPatternLegacy(names, " a.* ") === Seq("a1", "a2")) + assert(filterPatternLegacy(names, " B.*|a* ") === Seq("a1", "a2", "b2")) + assert(filterPatternLegacy(names, " a. ") === Seq("a1", "a2")) + assert(filterPatternLegacy(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("filter like pattern") { + 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 8c2b0d413b79d..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,33 +97,41 @@ class CatalogSuite extends SparkFunSuite { assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) } - test("listTables by pattern") { + 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") - intercept[NoSuchNamespaceException](catalog.listTables(Array("ns", "*test*"))) + 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) + catalog.createTable(ident1, schema, emptyTrans, emptyProps) - assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident1)) - intercept[NoSuchNamespaceException](catalog.listTables(Array("ns2"), "*test*")) + 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) + catalog.createTable(ident3, schema, emptyTrans, emptyProps) + catalog.createTable(ident2, schema, emptyTrans, emptyProps) - assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident1, ident2)) - assert(catalog.listTables(Array("ns2"), "*test*").toSet == Set(ident3)) + assert(catalog.listTables(Array("ns"), v._2).toSet == Set(ident1, ident2)) + assert(catalog.listTables(Array("ns2"), v._2).toSet == Set(ident3)) - catalog.dropTable(ident1) + catalog.dropTable(ident1) - assert(catalog.listTables(Array("ns"), "*test*").toSet == Set(ident2)) + assert(catalog.listTables(Array("ns"), v._2).toSet == Set(ident2)) - catalog.dropTable(ident2) + catalog.dropTable(ident2) + + assert(catalog.listTables(Array("ns"), v._2).isEmpty) + assert(catalog.listTables(Array("ns2"), v._2).toSet == Set(ident3)) - assert(catalog.listTables(Array("ns"), "*test*").isEmpty) - assert(catalog.listTables(Array("ns2"), "*test*").toSet == Set(ident3)) + catalog.dropNamespace(Array("ns"), cascade = true) + catalog.dropNamespace(Array("ns2"), cascade = true) + } + } } test("createTable") { 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..5ee62d77912dd 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.getAllMatchWildcard)) .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.getAllMatchWildcard))) .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 3a761541a00e2..c84d39e81daab 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.getAllMatchWildcard)) 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..22870cee566e5 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.getAllMatchWildcard)) } 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/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index f21c0c2b52fab..760aef23525c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2824,10 +2824,10 @@ class DataSourceV2SQLSuiteV1Filter assert(sql("SHOW CATALOGS").collect() === Array( Row("spark_catalog"), Row("testcat"), Row("testcat2"), Row("testpart"))) - assert(sql("SHOW CATALOGS LIKE 'test*'").collect() === Array( + assert(sql("SHOW CATALOGS LIKE 'test%'").collect() === Array( Row("testcat"), Row("testcat2"), Row("testpart"))) - assert(sql("SHOW CATALOGS LIKE 'testcat*'").collect() === Array( + assert(sql("SHOW CATALOGS LIKE 'testcat%'").collect() === Array( Row("testcat"), Row("testcat2"))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsSuiteBase.scala index 19bd830500834..eceb99890bfd7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsSuiteBase.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -138,10 +139,10 @@ trait ShowFunctionsSuiteBase extends QueryTest with DDLCommandTestUtils { assert(sql(s"SHOW USER FUNCTIONS IN $ns").isEmpty) funs.foreach(createFunction) QueryTest.checkAnswer( - sql(s"SHOW USER FUNCTIONS IN $ns LIKE '*'"), + sql(s"SHOW USER FUNCTIONS IN $ns LIKE '%'"), testFuns.map(testFun => Row(qualifiedFunName("ns", testFun)))) QueryTest.checkAnswer( - sql(s"SHOW USER FUNCTIONS IN $ns LIKE '*rc*'"), + sql(s"SHOW USER FUNCTIONS IN $ns LIKE '%rc%'"), Seq("crc32i", "crc16j").map(testFun => Row(qualifiedFunName("ns", testFun)))) } } @@ -159,15 +160,18 @@ trait ShowFunctionsSuiteBase extends QueryTest with DDLCommandTestUtils { test("show functions matched to the '|' pattern") { val testFuns = Seq("crc32i", "crc16j", "date1900", "Date1") - withNamespaceAndFuns("ns", testFuns) { (ns, funs) => - assert(sql(s"SHOW USER FUNCTIONS IN $ns").isEmpty) - funs.foreach(createFunction) - QueryTest.checkAnswer( - sql(s"SHOW USER FUNCTIONS IN $ns LIKE 'crc32i|date1900'"), - Seq("crc32i", "date1900").map(testFun => Row(qualifiedFunName("ns", testFun)))) - QueryTest.checkAnswer( - sql(s"SHOW USER FUNCTIONS IN $ns LIKE 'crc32i|date*'"), - Seq("crc32i", "date1900", "Date1").map(testFun => Row(qualifiedFunName("ns", testFun)))) + withSQLConf( + SQLConf.LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_PATTERN.key -> "true") { + withNamespaceAndFuns("ns", testFuns) { (ns, funs) => + assert(sql(s"SHOW USER FUNCTIONS IN $ns").isEmpty) + funs.foreach(createFunction) + QueryTest.checkAnswer( + sql(s"SHOW USER FUNCTIONS IN $ns LIKE 'crc32i|date1900'"), + Seq("crc32i", "date1900").map(testFun => Row(qualifiedFunName("ns", testFun)))) + QueryTest.checkAnswer( + sql(s"SHOW USER FUNCTIONS IN $ns LIKE 'crc32i|date*'"), + Seq("crc32i", "date1900", "Date1").map(testFun => Row(qualifiedFunName("ns", testFun)))) + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala index 80e545f6e3c2f..1221d5e8dbdb1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala @@ -85,7 +85,7 @@ trait ShowNamespacesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE NAMESPACE ns2") runShowNamespacesSql("SHOW NAMESPACES", Seq("ns1", "ns2") ++ builtinTopNamespaces) - runShowNamespacesSql("SHOW NAMESPACES LIKE '*1*'", Seq("ns1")) + runShowNamespacesSql("SHOW NAMESPACES LIKE '%1%'", Seq("ns1")) } } } @@ -96,9 +96,9 @@ trait ShowNamespacesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE NAMESPACE $catalog.showdb1a") Seq( - "'*db1A'" -> Seq("showdb1a"), - "'*2*'" -> Seq("showdb2b"), - "'*db1A|*db2B'" -> Seq("showdb1a", "showdb2b") + "'%db1A'" -> Seq("showdb1a"), + "'%2%'" -> Seq("showdb2b"), + "'%db__'" -> Seq("showdb1a", "showdb2b") ).foreach { case (pattern, expected) => runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE $pattern", expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index c88217221ab76..76871a520e682 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -105,19 +105,19 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { Row("ns1", "table_name_2b", false))) runShowTablesSql( - s"SHOW TABLES FROM $catalog.ns1 LIKE '*name*'", + s"SHOW TABLES FROM $catalog.ns1 LIKE '%name%'", Seq( Row("ns1", "table_name_1a", false), Row("ns1", "table_name_2b", false))) runShowTablesSql( - s"SHOW TABLES FROM $catalog.ns1 LIKE 'table_name_1*|table_name_2*'", + s"SHOW TABLES FROM $catalog.ns1 LIKE 'table\\_name\\_%'", Seq( Row("ns1", "table_name_1a", false), Row("ns1", "table_name_2b", false))) runShowTablesSql( - s"SHOW TABLES FROM $catalog.ns1 LIKE '*2b'", + s"SHOW TABLES FROM $catalog.ns1 LIKE '%2b'", Seq(Row("ns1", "table_name_2b", false))) } } @@ -175,7 +175,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { test("show table extended in a not existing namespace") { checkError( exception = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.nonexist LIKE '*tbl*'") + sql(s"SHOW TABLE EXTENDED IN $catalog.nonexist LIKE '%tbl%'") }, errorClass = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`nonexist`")) @@ -185,7 +185,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val namespace = "ns1" val table = "nonexist" withNamespaceAndTable(namespace, table, catalog) { _ => - val result = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '*$table*'") + val result = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '%$table%'") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result.collect().isEmpty) @@ -284,8 +284,8 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE TABLE $catalog.$namespace.$table2 (data2 string, id2 bigint) " + s"$defaultUsing PARTITIONED BY (id2)") - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") - .sort("tableName") + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table%'"). + sort("tableName") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) val resultCollect = result.collect() @@ -335,7 +335,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE GLOBAL TEMPORARY VIEW $globalTmpViewName AS SELECT id FROM $t") // temp local view - val localResult = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") + val localResult = sql(s"SHOW TABLE EXTENDED LIKE '$viewName%'").sort("tableName") assert(localResult.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) val localResultCollect = localResult.collect() @@ -358,7 +358,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { assert(actualLocalResult === expectedLocalResult) // temp global view - val globalResult = sql(s"SHOW TABLE EXTENDED IN global_temp LIKE '$viewName*'"). + val globalResult = sql(s"SHOW TABLE EXTENDED IN global_temp LIKE '$viewName%'"). sort("tableName") assert(globalResult.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) @@ -443,7 +443,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val result1 = sql(s"SELECT * FROM $catalog.$namespace.$table1") assert(result1.schema.fieldNames === selectCommandSchema) - val extendedResult = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table*'"). + val extendedResult = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table%'"). sort("tableName") val extendedResultCollect = extendedResult.collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index 4b4742910bd18..1fbcc1265223a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -65,11 +65,12 @@ trait ShowTablesSuiteBase extends command.ShowTablesSuiteBase with command.Tests withSourceViews { val expected = Seq(Row("", "source", true), Row("", "source2", true)) - val df = sql("SHOW TABLE EXTENDED FROM default LIKE '*source*'") + val df = sql("SHOW TABLE EXTENDED FROM default LIKE '%source%'") + val result = df.collect() val resultWithoutInfo = result.map { case Row(db, table, temp, _) => Row(db, table, temp) } - assert(resultWithoutInfo === expected) + assert(resultWithoutInfo.sortBy(r => r.getString(1)) === expected) result.foreach { case Row(_, _, _, info: String) => assert(info.nonEmpty) } } } @@ -186,7 +187,7 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { val viewName = table + "_view" withView(viewName) { sql(s"CREATE VIEW $catalog.$namespace.$viewName AS SELECT id FROM $t") - val result = sql(s"SHOW TABLE EXTENDED in $namespace LIKE '$viewName*'").sort("tableName") + val result = sql(s"SHOW TABLE EXTENDED in $namespace LIKE '$viewName%'").sort("tableName") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) val resultCollect = result.collect() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 84f75e3ef5036..3aa5d4fc87b42 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils, StringUtils} import org.apache.spark.sql.catalyst.util.TypeUtils.{toSQLId, toSQLValue} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{PartitioningUtils, SourceOptions} @@ -237,7 +237,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } override def listDatabases(): Seq[String] = withClient { - client.listDatabases("*") + client.listDatabases(StringUtils.getAllMatchWildcard) } override def listDatabases(pattern: String): Seq[String] = withClient { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 9943c0178fcf3..02581e5aff38f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTable, CatalogTablePartition, CatalogUtils, ExternalCatalogUtils, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateFormatter, TypeUtils} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateFormatter, StringUtils, TypeUtils} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.internal.SQLConf @@ -626,7 +626,12 @@ private[client] class Shim_v2_0 extends Shim with Logging { override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { recordHiveCall() - hive.getFunctions(db, pattern).asScala.toSeq + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + hive.getFunctions(db, pattern).asScala.toSeq + } else { + val functions = hive.getFunctions(db, "*").asScala.toSeq + StringUtils.filterBySQLLikePattern(functions, pattern) + } } /** @@ -904,7 +909,12 @@ private[client] class Shim_v2_0 extends Shim with Logging { override def getDatabasesByPattern(hive: Hive, pattern: String): Seq[String] = { recordHiveCall() - hive.getDatabasesByPattern(pattern).asScala.toSeq + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + hive.getDatabasesByPattern(pattern).asScala.toSeq + } else { + val databases = hive.getDatabasesByPattern("*").asScala.toSeq + StringUtils.filterBySQLLikePattern(databases, pattern) + } } override def databaseExists(hive: Hive, dbName: String): Boolean = { @@ -933,7 +943,12 @@ private[client] class Shim_v2_0 extends Shim with Logging { override def getTablesByPattern(hive: Hive, dbName: String, pattern: String): Seq[String] = { recordHiveCall() - hive.getTablesByPattern(dbName, pattern).asScala.toSeq + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + hive.getTablesByPattern(dbName, pattern).asScala.toSeq + } else { + val tables = hive.getTablesByPattern(dbName, "*").asScala.toSeq + StringUtils.filterBySQLLikePattern(tables, pattern) + } } override def getAllTables(hive: Hive, dbName: String): Seq[String] = { @@ -1131,8 +1146,14 @@ private[client] class Shim_v2_3 extends Shim_v2_1 { tableType: TableType): Seq[String] = { recordHiveCall() try { - getTablesByTypeMethod.invoke(hive, dbName, pattern, tableType) - .asInstanceOf[JList[String]].asScala.toSeq + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + getTablesByTypeMethod.invoke(hive, dbName, pattern, tableType) + .asInstanceOf[JList[String]].asScala.toSeq + } else { + val tables = getTablesByTypeMethod.invoke(hive, dbName, "*", tableType) + .asInstanceOf[JList[String]].asScala.toSeq + StringUtils.filterBySQLLikePattern(tables, pattern) + } } catch { case ex: InvocationTargetException if ex.getCause.isInstanceOf[HiveException] => val cause = ex.getCause.getCause diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 5502414629c01..006daff3ac721 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -870,9 +870,9 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_))) if (analyzedByHive) { - assert(StringUtils.filterPattern(describeResult1, "*numRows\\s+500*").nonEmpty) + assert(StringUtils.filterPatternLegacy(describeResult1, "*numRows\\s+500*").nonEmpty) } else { - assert(StringUtils.filterPattern(describeResult1, "*numRows\\s+500*").isEmpty) + assert(StringUtils.filterPatternLegacy(describeResult1, "*numRows\\s+500*").isEmpty) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 6ecb611699f1e..58fd4d29e3be7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -78,48 +78,51 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("show views") { - withView("show1a", "show2b", "global_temp.temp1", "temp2") { - sql("CREATE VIEW show1a AS SELECT 1 AS id") - sql("CREATE VIEW show2b AS SELECT 1 AS id") - sql("CREATE GLOBAL TEMP VIEW temp1 AS SELECT 1 AS id") - sql("CREATE TEMP VIEW temp2 AS SELECT 1 AS id") - checkAnswer( - sql("SHOW VIEWS"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: - Row("default", "parquet_view1", false) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS IN default"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: - Row("default", "parquet_view1", false) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS FROM default"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: - Row("default", "parquet_view1", false) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS FROM global_temp"), - Row("global_temp", "temp1", true) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS 'show1*|show2*'"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: Nil) - checkAnswer( - sql("SHOW VIEWS LIKE 'show1*|show2*'"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: Nil) - checkAnswer( - sql("SHOW VIEWS IN default 'show1*'"), - Row("default", "show1a", false) :: Nil) - checkAnswer( - sql("SHOW VIEWS IN default LIKE 'show1*|show2*'"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: Nil) + withSQLConf( + SQLConf.LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_PATTERN.key -> "true") { + withView("show1a", "show2b", "global_temp.temp1", "temp2") { + sql("CREATE VIEW show1a AS SELECT 1 AS id") + sql("CREATE VIEW show2b AS SELECT 1 AS id") + sql("CREATE GLOBAL TEMP VIEW temp1 AS SELECT 1 AS id") + sql("CREATE TEMP VIEW temp2 AS SELECT 1 AS id") + checkAnswer( + sql("SHOW VIEWS"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS FROM default"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS FROM global_temp"), + Row("global_temp", "temp1", true) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS LIKE 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default 'show1*'"), + Row("default", "show1a", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default LIKE 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 79b1eb6c0961a..7c81d71fc49de 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -77,7 +77,7 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { val viewName = table + "_view" withView(viewName) { sql(s"CREATE VIEW $catalog.$namespace.$viewName AS SELECT id FROM $t") - val result = sql(s"SHOW TABLE EXTENDED in $namespace LIKE '$viewName*'").sort("tableName") + val result = sql(s"SHOW TABLE EXTENDED in $namespace LIKE '$viewName%'").sort("tableName") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) val resultCollect = result.collect() From 4d382ef2e5b39fcb26512a7276045c4bcd72ae88 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 26 Feb 2024 19:02:00 +0800 Subject: [PATCH 07/11] fix some UT --- .../org/apache/spark/sql/CatalogSuite.scala | 16 ++--- python/pyspark/sql/tests/test_catalog.py | 16 ++--- .../catalog/ExternalCatalogSuite.scala | 24 +++---- .../catalog/SessionCatalogSuite.scala | 66 +++++++++---------- .../v2/jdbc/JDBCTableCatalogSuite.scala | 2 +- .../spark/sql/internal/CatalogSuite.scala | 34 +++++----- .../org/apache/spark/sql/hive/UDFSuite.scala | 2 +- .../sql/hive/client/HiveClientSuite.scala | 4 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 9 files changed, 83 insertions(+), 83 deletions(-) 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/python/pyspark/sql/tests/test_catalog.py b/python/pyspark/sql/tests/test_catalog.py index 278fbbb2ba510..07458d11a88df 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/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/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index 0088fab7d209e..fe0b63a9571a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -569,7 +569,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { test("SPARK-42904: CREATE TABLE with char/varchar") { withTable("h2.test.new_table") { sql("CREATE TABLE h2.test.new_table(c CHAR(10), v VARCHAR(100))") - checkAnswer(sql("SHOW TABLES IN h2.test LIKE 'new*'"), Row("test", "new_table", false)) + checkAnswer(sql("SHOW TABLES IN h2.test LIKE 'new%'"), Row("test", "new_table", false)) } } 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 15733d1c8bf64..4fe7e09981ce5 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 @@ -158,9 +158,9 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf createDatabase("my_db2") assert(spark.catalog.listDatabases().collect().map(_.name).toSet == Set("default", "my_db1", "my_db2")) - assert(spark.catalog.listDatabases("my*").collect().map(_.name).toSet == + assert(spark.catalog.listDatabases("my%").collect().map(_.name).toSet == Set("my_db1", "my_db2")) - assert(spark.catalog.listDatabases("you*").collect().map(_.name).toSet == + assert(spark.catalog.listDatabases("you%").collect().map(_.name).toSet == Set.empty[String]) dropDatabase("my_db1") assert(spark.catalog.listDatabases().collect().map(_.name).toSet == @@ -182,18 +182,18 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf // requiring the pattern to be quoted. This is not ideal, we should fix it in the future. if (legacy) { assert( - spark.catalog.listDatabases("my*").collect().map(_.name).toSet == + spark.catalog.listDatabases("my%").collect().map(_.name).toSet == Set("`my-db1`", "`my``db2`") ) - assert(spark.catalog.listDatabases("`my*`").collect().map(_.name).toSet == Set.empty) + assert(spark.catalog.listDatabases("`my%`").collect().map(_.name).toSet == Set.empty) } else { - assert(spark.catalog.listDatabases("my*").collect().map(_.name).toSet == Set.empty) + assert(spark.catalog.listDatabases("my%").collect().map(_.name).toSet == Set.empty) assert( - spark.catalog.listDatabases("`my*`").collect().map(_.name).toSet == + spark.catalog.listDatabases("`my%`").collect().map(_.name).toSet == Set("`my-db1`", "`my``db2`") ) } - assert(spark.catalog.listDatabases("you*").collect().map(_.name).toSet == + assert(spark.catalog.listDatabases("you%").collect().map(_.name).toSet == Set.empty[String]) dropDatabase("my-db1") assert(spark.catalog.listDatabases().collect().map(_.name).toSet == @@ -222,16 +222,16 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf createTempTable("my_temp_table") assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_table1", "my_table2", "my_temp_table")) - assert(spark.catalog.listTables(spark.catalog.currentDatabase, "my_table*").collect() + assert(spark.catalog.listTables(spark.catalog.currentDatabase, "my_table%").collect() .map(_.name).toSet == Set("my_table1", "my_table2")) dropTable("my_table1") assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_table2", "my_temp_table")) - assert(spark.catalog.listTables(spark.catalog.currentDatabase, "my_table*").collect() + assert(spark.catalog.listTables(spark.catalog.currentDatabase, "my_table%").collect() .map(_.name).toSet == Set("my_table2")) dropTable("my_temp_table") assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_table2")) - assert(spark.catalog.listTables(spark.catalog.currentDatabase, "my_table*").collect() + assert(spark.catalog.listTables(spark.catalog.currentDatabase, "my_table%").collect() .map(_.name).toSet == Set("my_table2")) } @@ -273,12 +273,12 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf Set("my_table1", "my_temp_table")) assert(spark.catalog.listTables("my_db2").collect().map(_.name).toSet == Set("my_table2", "my_temp_table")) - assert(spark.catalog.listTables("my_db2", "my_table*").collect().map(_.name).toSet == + assert(spark.catalog.listTables("my_db2", "my_table%").collect().map(_.name).toSet == Set("my_table2")) dropTable("my_table1", Some("my_db1")) assert(spark.catalog.listTables("my_db1").collect().map(_.name).toSet == Set("my_temp_table")) - assert(spark.catalog.listTables("my_db1", "my_table*").collect().isEmpty) + assert(spark.catalog.listTables("my_db1", "my_table%").collect().isEmpty) assert(spark.catalog.listTables("my_db2").collect().map(_.name).toSet == Set("my_table2", "my_temp_table")) dropTable("my_temp_table") @@ -303,7 +303,7 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf assert(funcNames1.contains("my_func2")) assert(funcNames1.contains("my_temp_func")) val funcNamesWithPattern1 = - spark.catalog.listFunctions("default", "my_func*").collect().map(_.name).toSet + spark.catalog.listFunctions("default", "my_func%").collect().map(_.name).toSet assert(funcNamesWithPattern1.contains("my_func1")) assert(funcNamesWithPattern1.contains("my_func2")) assert(!funcNamesWithPattern1.contains("my_temp_func")) @@ -314,12 +314,12 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf assert(funcNames2.contains("my_func2")) assert(!funcNames2.contains("my_temp_func")) val funcNamesWithPattern2 = - spark.catalog.listFunctions("default", "my_func*").collect().map(_.name).toSet + spark.catalog.listFunctions("default", "my_func%").collect().map(_.name).toSet assert(!funcNamesWithPattern2.contains("my_func1")) assert(funcNamesWithPattern2.contains("my_func2")) assert(!funcNamesWithPattern2.contains("my_temp_func")) val funcNamesWithPattern3 = - spark.catalog.listFunctions("default", "*not_existing_func*").collect().map(_.name).toSet + spark.catalog.listFunctions("default", "%not_existing_func%").collect().map(_.name).toSet assert(funcNamesWithPattern3.isEmpty) } @@ -947,9 +947,9 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf assert(spark.catalog.currentCatalog().equals("spark_catalog")) assert(spark.catalog.listCatalogs().collect().map(c => c.name).toSet == Set("testcat", CatalogManager.SESSION_CATALOG_NAME)) - assert(spark.catalog.listCatalogs("spark*").collect().map(c => c.name).toSet == + assert(spark.catalog.listCatalogs("spark%").collect().map(c => c.name).toSet == Set(CatalogManager.SESSION_CATALOG_NAME)) - assert(spark.catalog.listCatalogs("spark2*").collect().map(c => c.name).toSet == + assert(spark.catalog.listCatalogs("spark2%").collect().map(c => c.name).toSet == Set.empty) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala index 5dce214a896be..ea997dbf01710 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -95,7 +95,7 @@ class UDFSuite withUserDefinedFunction(functionName -> false) { sql(s"CREATE FUNCTION $functionName AS '$functionClass'") checkAnswer( - sql("SHOW functions like '.*upper'"), + sql("SHOW functions like '%myupper'"), Row(s"$SESSION_CATALOG_NAME.default.$functionNameLower") ) checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala index 0bc288501a01e..efe38dc5aac0b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -152,7 +152,7 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) { } test("listDatabases") { - assert(client.listDatabases("defau.*") == Seq("default")) + assert(client.listDatabases("defau%") == Seq("default")) } test("alterDatabase") { @@ -539,7 +539,7 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) { } test("listFunctions") { - assert(client.listFunctions("default", "fun.*").size == 1) + assert(client.listFunctions("default", "fun%").size == 1) } test("dropFunction") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 7b0e0b28780ca..de9f864b30a45 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1766,7 +1766,7 @@ class HiveDDLSuite client.runSqlHive( s"CREATE INDEX $indexName ON TABLE $tabName (a) AS 'COMPACT' WITH DEFERRED REBUILD") val indexTabName = - spark.sessionState.catalog.listTables("default", s"*$indexName*").head.table + spark.sessionState.catalog.listTables("default", s"%$indexName%").head.table // Even if index tables exist, listTables APIs should still work checkAnswer( From 677f2c971e460d70bd9a3940d1dccb592a606409 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 27 Feb 2024 16:18:18 +0800 Subject: [PATCH 08/11] fix ut --- python/pyspark/sql/catalog.py | 16 ++++---- .../analyzer-results/show-tables.sql.out | 26 ++++++------- .../analyzer-results/show-views.sql.out | 18 ++++----- .../sql-tests/inputs/show-tables.sql | 12 +++--- .../resources/sql-tests/inputs/show-views.sql | 10 ++--- .../sql-tests/results/show-tables.sql.out | 17 +++++---- .../sql-tests/results/show-views.sql.out | 11 +++--- .../cli/operation/MetadataOperation.java | 37 +++++++++++++++++++ .../SparkGetColumnsOperation.scala | 15 ++++++-- .../SparkGetFunctionsOperation.scala | 11 ++++-- .../SparkGetSchemasOperation.scala | 8 +++- .../SparkGetTablesOperation.scala | 9 ++++- 12 files changed, 127 insertions(+), 63 deletions(-) diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 6595659a4daea..a3a0866a0dd81 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/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 -- !query output @@ -86,16 +86,17 @@ show_t3 -- !query -SHOW TABLES LIKE 'show_t1*|show_t2*' +SHOW TABLES LIKE 'show_t%' -- !query schema struct -- !query output show_t1 show_t2 +show_t3 -- !query -SHOW TABLES IN showdb 'show_t*' +SHOW TABLES IN showdb 'show_t%' -- !query schema struct -- !query output @@ -105,7 +106,7 @@ show_t3 -- !query -SHOW TABLES IN showdb LIKE 'show_t*' +SHOW TABLES IN showdb LIKE 'show_t%' -- !query schema struct -- !query output @@ -115,7 +116,7 @@ show_t3 -- !query -SHOW TABLE EXTENDED LIKE 'show_t*' +SHOW TABLE EXTENDED LIKE 'show_t%' -- !query schema struct -- !query output @@ -204,7 +205,7 @@ 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 schema struct<> -- !query output @@ -213,14 +214,14 @@ 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/results/show-views.sql.out b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out index bfed13683d9dd..7d7bad6f58a6a 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-views.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out @@ -95,7 +95,7 @@ view_4 -- !query -SHOW VIEWS 'view_*' +SHOW VIEWS 'view_%' -- !query schema struct -- !query output @@ -105,16 +105,17 @@ view_4 -- !query -SHOW VIEWS LIKE 'view_1*|view_2*' +SHOW VIEWS LIKE 'view_%' -- !query schema struct -- !query output view_1 view_2 +view_4 -- !query -SHOW VIEWS IN showdb 'view_*' +SHOW VIEWS IN showdb 'view_%' -- !query schema struct -- !query output @@ -124,7 +125,7 @@ view_4 -- !query -SHOW VIEWS IN showdb LIKE 'view_*' +SHOW VIEWS IN showdb LIKE 'view_%' -- !query schema struct -- !query output @@ -134,7 +135,7 @@ view_4 -- !query -SHOW VIEWS IN wrongdb LIKE 'view_*' +SHOW VIEWS IN wrongdb LIKE 'view_%' -- !query schema struct<> -- !query output diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java index a818d1ecfd16e..ac063c50e3a68 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -105,6 +105,43 @@ private String convertPattern(final String pattern, boolean datanucleusFormat) { .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); } + protected String newConvertIdentifierPattern(final String pattern, boolean datanucleusFormat) { + if (pattern == null) { + return newConvertPattern("%", true); + } else { + return newConvertPattern(pattern, datanucleusFormat); + } + } + + public String newConvertSchemaPattern(final String pattern, boolean datanucleusFormat) { + if ((pattern == null) || pattern.isEmpty()) { + String all = datanucleusFormat? "*" : ".*"; + return newConvertPattern(all, datanucleusFormat); + } else { + return newConvertPattern(pattern, datanucleusFormat); + } + } + + public String newConvertPattern(final String pattern, boolean datanucleusFormat) { + if (datanucleusFormat) { + return pattern + .replaceAll("([^\\\\])\\*", "$1%") + .replaceAll("\\\\\\*", "*") + .replaceAll("^\\*", "%") + .replaceAll("([^\\\\])\\.", "$1_") + .replaceAll("\\\\\\.", ".") + .replaceAll("^\\.", "_"); + } else { + return pattern + .replaceAll("([^\\\\])\\.\\*", "$1%") + .replaceAll("\\\\\\.\\*", "*") + .replaceAll("^\\.\\*", "%") + .replaceAll("([^\\\\])\\.", "$1_") + .replaceAll("\\\\\\.", ".") + .replaceAll("^\\.", "_"); + } + } + protected boolean isAuthV2Enabled(){ SessionState ss = SessionState.get(); return (ss.isAuthorizationModeV2() && diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 5dd8caf3f221d..d444fd10a3808 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -74,12 +75,20 @@ private[hive] class SparkGetColumnsOperation( statementId, parentSession.getUsername) - val schemaPattern = convertSchemaPattern(schemaName) - val tablePattern = convertIdentifierPattern(tableName, true) + val (schemaPattern, tablePattern) = + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + (convertSchemaPattern(schemaName), convertIdentifierPattern(tableName, true)) + } else { + (newConvertSchemaPattern(schemaName, true), newConvertIdentifierPattern(tableName, true)) + } var columnPattern: Pattern = null if (columnName != null) { - columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)) + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)) + } else { + columnPattern = Pattern.compile(newConvertIdentifierPattern(columnName, false)) + } } val db2Tabs = catalog.listDatabases(schemaPattern).map { dbName => diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 53a94a128c0ef..19ef391a5031a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -29,6 +29,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.internal.SQLConf /** * Spark's own GetFunctionsOperation @@ -61,9 +62,13 @@ private[hive] class SparkGetFunctionsOperation( val catalog = sqlContext.sessionState.catalog // get databases for schema pattern - val schemaPattern = convertSchemaPattern(schemaName) + val (schemaPattern, functionPattern) = + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + (convertSchemaPattern(schemaName), convertSchemaPattern(functionName)) + } else { + (newConvertSchemaPattern(schemaName, true), newConvertSchemaPattern(functionName, true)) + } val matchingDbs = catalog.listDatabases(schemaPattern) - val functionPattern = CLIServiceUtils.patternToRegex(functionName) if (isAuthV2Enabled) { // authorize this call on the schema objects @@ -81,7 +86,7 @@ private[hive] class SparkGetFunctionsOperation( try { matchingDbs.foreach { db => - catalog.listFunctions(db, functionPattern).foreach { + catalog.listFunctions(db, functionPattern).sortBy { item => item._1.funcName }.foreach { case (funcIdentifier, _) => val info = catalog.lookupFunctionInfo(funcIdentifier) val rowData = Array[AnyRef]( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 45cfa86ba9343..b5fe5001e9351 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -27,6 +27,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.internal.SQLConf /** * Spark's own GetSchemasOperation @@ -67,7 +68,12 @@ private[hive] class SparkGetSchemasOperation( parentSession.getUsername) try { - val schemaPattern = convertSchemaPattern(schemaName) + val schemaPattern = + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + convertSchemaPattern(schemaName) + } else { + newConvertSchemaPattern(schemaName, true) + } sqlContext.sessionState.catalog.listDatabases(schemaPattern).foreach { dbName => rowSet.addRow(Array[AnyRef](dbName, DEFAULT_HIVE_CATALOG)) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 38aaed0be2148..f14f9a973d0de 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -30,6 +30,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ +import org.apache.spark.sql.internal.SQLConf /** * Spark's own GetTablesOperation @@ -64,8 +65,12 @@ private[hive] class SparkGetTablesOperation( Thread.currentThread().setContextClassLoader(executionHiveClassLoader) val catalog = sqlContext.sessionState.catalog - val schemaPattern = convertSchemaPattern(schemaName) - val tablePattern = convertIdentifierPattern(tableName, true) + val (schemaPattern, tablePattern) = + if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { + (convertSchemaPattern(schemaName), convertIdentifierPattern(tableName, true)) + } else { + (newConvertSchemaPattern(schemaName, true), newConvertIdentifierPattern(tableName, true)) + } val matchingDbs = catalog.listDatabases(schemaPattern) if (isAuthV2Enabled) { From 4b096f2d972bfbb5e89b2e5589bc1b5fefde07f3 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 27 Feb 2024 19:35:22 +0800 Subject: [PATCH 09/11] update comment & doc & ut --- docs/sql-ref-syntax-aux-show-tables.md | 11 +- .../sql/connector/catalog/TableCatalog.java | 8 +- .../spark/sql/catalyst/util/StringUtils.scala | 16 +-- .../cli/operation/GetColumnsOperation.java | 22 +++- .../cli/operation/GetFunctionsOperation.java | 9 +- .../cli/operation/GetSchemasOperation.java | 9 +- .../cli/operation/GetTablesOperation.java | 14 ++- .../cli/operation/MetadataOperation.java | 85 -------------- .../cli/operation/MetadataOperationUtils.java | 107 ++++++++++++++++++ .../SparkGetColumnsOperation.scala | 1 + .../SparkGetFunctionsOperation.scala | 1 + .../SparkGetSchemasOperation.scala | 1 + .../SparkGetTablesOperation.scala | 1 + .../MetadataOperationUtilsSuite.scala | 51 +++++++++ 14 files changed, 230 insertions(+), 106 deletions(-) create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md index fef9722a444f8..f53d1ed145c4d 100644 --- a/docs/sql-ref-syntax-aux-show-tables.md +++ b/docs/sql-ref-syntax-aux-show-tables.md @@ -40,12 +40,18 @@ 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. + After Spark Version 4.0 + * We accept only SQL type like expressions containing '%' for any character(s), and '_' for a single character. + * Examples are 'employees', 'emp%', 'emplo_ees', all of which will match the database named 'employees'. + + Before Spark 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 +85,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 +94,7 @@ SHOW TABLES FROM default LIKE 'sam*'; +--------+---------+-----------+ -- List all tables matching the pattern `sam*|suj` +Note: After Spark 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/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 f02ab109fe70c..1745d0e047c82 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 @@ -108,9 +108,11 @@ public interface TableCatalog extends CatalogPlugin { * 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, only '*' and '|' are allowed as wildcards, others will - * follow regular expression convention, case-insensitive match and white spaces - * on both ends will be ignored. Please refer to regex_pattern in https:// + * @param pattern the filter pattern, + * when SQLConf.LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_PATTERN + * 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 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 bcdf19b78fc98..6d223c91da89a 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 @@ -124,7 +124,8 @@ object StringUtils extends Logging { } /** - * This utility can be used for filtering pattern in the "Like" of "Show Tables / Functions" DDL + * 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 @@ -144,6 +145,13 @@ 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, 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) @@ -153,12 +161,6 @@ object StringUtils extends Logging { } } - /** - * - * @param pattern the filter pattern, only '%' and '_' are allowed as wildcards, others will - * follow regular expression convention, - * @return the filtered names list - */ private[util] def likePatternToRegExp(pattern: String): String = { val regExp = new StringBuilder() diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java index 250adc51f81e9..acf579ca3f5fa 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -51,6 +51,8 @@ import org.apache.hive.service.rpc.thrift.TRowSet; import org.apache.hive.service.rpc.thrift.TTableSchema; +import org.apache.spark.sql.internal.SQLConf; + /** * GetColumnsOperation. * @@ -134,12 +136,22 @@ public void runInternal() throws HiveSQLException { setState(OperationState.RUNNING); try { IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - String tablePattern = convertIdentifierPattern(tableName, true); - + String schemaPattern, tablePattern; Pattern columnPattern = null; - if (columnName != null) { - columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)); + if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); + tablePattern = MetadataOperationUtils.convertIdentifierPattern(tableName, true); + if (columnName != null) { + columnPattern = Pattern.compile( + MetadataOperationUtils.convertIdentifierPattern(columnName, false)); + } + } else { + schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); + tablePattern = MetadataOperationUtils.newConvertIdentifierPattern(tableName, true); + if (columnName != null) { + columnPattern = Pattern.compile( + MetadataOperationUtils.newConvertIdentifierPattern(columnName, false)); + } } List dbNames = metastoreClient.getDatabases(schemaPattern); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java index 3f02f753bf875..7611d0b85834d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -41,6 +41,8 @@ import org.apache.hive.service.rpc.thrift.TTableSchema; import org.apache.thrift.TException; +import org.apache.spark.sql.internal.SQLConf; + /** * GetFunctionsOperation. * @@ -81,7 +83,12 @@ public void runInternal() throws HiveSQLException { if (isAuthV2Enabled()) { // get databases for schema pattern IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); + String schemaPattern; + if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); + } else { + schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); + } List matchingDbs; try { matchingDbs = metastoreClient.getDatabases(schemaPattern); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index 865e264bd5f4f..9962236ddf2b0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -30,6 +30,8 @@ import org.apache.hive.service.rpc.thrift.TRowSet; import org.apache.hive.service.rpc.thrift.TTableSchema; +import org.apache.spark.sql.internal.SQLConf; + /** * GetSchemasOperation. * @@ -61,7 +63,12 @@ public void runInternal() throws HiveSQLException { } try { IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); + String schemaPattern; + if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); + } else { + schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); + } for (String dbName : metastoreClient.getDatabases(schemaPattern)) { rowSet.addRow(new Object[] {dbName, DEFAULT_HIVE_CATALOG}); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java index bd9f0814814f1..b8bd50687aa17 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -38,6 +38,8 @@ import org.apache.hive.service.rpc.thrift.TRowSet; import org.apache.hive.service.rpc.thrift.TTableSchema; +import org.apache.spark.sql.internal.SQLConf; + /** * GetTablesOperation. * @@ -93,7 +95,16 @@ public void runInternal() throws HiveSQLException { setState(OperationState.RUNNING); try { IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); + + String schemaPattern, tablePattern; + if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); + tablePattern = MetadataOperationUtils.convertIdentifierPattern(tableName, true); + } else { + schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); + tablePattern = MetadataOperationUtils.newConvertIdentifierPattern(tableName, true); + } + List matchingDbs = metastoreClient.getDatabases(schemaPattern); if(isAuthV2Enabled()){ List privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs); @@ -101,7 +112,6 @@ public void runInternal() throws HiveSQLException { authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr); } - String tablePattern = convertIdentifierPattern(tableName, true); for (TableMeta tableMeta : metastoreClient.getTableMeta(schemaPattern, tablePattern, tableTypeList)) { rowSet.addRow(new Object[] { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java index ac063c50e3a68..c5f0baa0f542b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -57,91 +57,6 @@ public void close() throws HiveSQLException { cleanupOperationLog(); } - /** - * Convert wildchars and escape sequence from JDBC format to datanucleous/regex - */ - protected String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { - if (pattern == null) { - return convertPattern("%", true); - } else { - return convertPattern(pattern, datanucleusFormat); - } - } - - /** - * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex - * The schema pattern treats empty string also as wildchar - */ - protected String convertSchemaPattern(final String pattern) { - if ((pattern == null) || pattern.isEmpty()) { - return convertPattern("%", true); - } else { - return convertPattern(pattern, true); - } - } - - /** - * Convert a pattern containing JDBC catalog search wildcards into - * Java regex patterns. - * - * @param pattern input which may contain '%' or '_' wildcard characters, or - * these characters escaped using {@link #getSearchStringEscape()}. - * @return replace %/_ with regex search characters, also handle escaped - * characters. - * - * The datanucleus module expects the wildchar as '*'. The columns search on the - * other hand is done locally inside the hive code and that requires the regex wildchar - * format '.*' This is driven by the datanucleusFormat flag. - */ - private String convertPattern(final String pattern, boolean datanucleusFormat) { - String wStr; - if (datanucleusFormat) { - wStr = "*"; - } else { - wStr = ".*"; - } - return pattern - .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr) - .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); - } - - protected String newConvertIdentifierPattern(final String pattern, boolean datanucleusFormat) { - if (pattern == null) { - return newConvertPattern("%", true); - } else { - return newConvertPattern(pattern, datanucleusFormat); - } - } - - public String newConvertSchemaPattern(final String pattern, boolean datanucleusFormat) { - if ((pattern == null) || pattern.isEmpty()) { - String all = datanucleusFormat? "*" : ".*"; - return newConvertPattern(all, datanucleusFormat); - } else { - return newConvertPattern(pattern, datanucleusFormat); - } - } - - public String newConvertPattern(final String pattern, boolean datanucleusFormat) { - if (datanucleusFormat) { - return pattern - .replaceAll("([^\\\\])\\*", "$1%") - .replaceAll("\\\\\\*", "*") - .replaceAll("^\\*", "%") - .replaceAll("([^\\\\])\\.", "$1_") - .replaceAll("\\\\\\.", ".") - .replaceAll("^\\.", "_"); - } else { - return pattern - .replaceAll("([^\\\\])\\.\\*", "$1%") - .replaceAll("\\\\\\.\\*", "*") - .replaceAll("^\\.\\*", "%") - .replaceAll("([^\\\\])\\.", "$1_") - .replaceAll("\\\\\\.", ".") - .replaceAll("^\\.", "_"); - } - } - protected boolean isAuthV2Enabled(){ SessionState ss = SessionState.get(); return (ss.isAuthorizationModeV2() && diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java new file mode 100644 index 0000000000000..903c6e1f18cba --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java @@ -0,0 +1,107 @@ +/* + * 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. + */ + +package org.apache.hive.service.cli.operation; + +public class MetadataOperationUtils { + + /** + * Convert wildchars and escape sequence from JDBC format to datanucleous/regex + */ + public static String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { + if (pattern == null) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, datanucleusFormat); + } + } + + /** + * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex + * The schema pattern treats empty string also as wildchar + */ + public static String convertSchemaPattern(final String pattern) { + if ((pattern == null) || pattern.isEmpty()) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, true); + } + } + + /** + * Convert a pattern containing JDBC catalog search wildcards into + * Java regex patterns. + * + * @param pattern input which may contain '%' or '_' wildcard characters, or + * these characters escaped using {@link #getSearchStringEscape()}. + * @return replace %/_ with regex search characters, also handle escaped + * characters. + * + * The datanucleus module expects the wildchar as '*'. The columns search on the + * other hand is done locally inside the hive code and that requires the regex wildchar + * format '.*' This is driven by the datanucleusFormat flag. + */ + public static String convertPattern(final String pattern, boolean datanucleusFormat) { + String wStr; + if (datanucleusFormat) { + wStr = "*"; + } else { + wStr = ".*"; + } + return pattern + .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr) + .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); + } + + public static String newConvertIdentifierPattern( + final String pattern, boolean datanucleusFormat) { + if (pattern == null) { + return newConvertPattern("%", true); + } else { + return newConvertPattern(pattern, datanucleusFormat); + } + } + + public static String newConvertSchemaPattern(final String pattern, boolean datanucleusFormat) { + if ((pattern == null) || pattern.isEmpty()) { + String all = datanucleusFormat? "*" : ".*"; + return newConvertPattern(all, datanucleusFormat); + } else { + return newConvertPattern(pattern, datanucleusFormat); + } + } + + public static String newConvertPattern(final String pattern, boolean datanucleusFormat) { + if (datanucleusFormat) { + return pattern + .replaceAll("([^\\\\])\\*", "$1%") + .replaceAll("\\\\\\*", "*") + .replaceAll("^\\*", "%") + .replaceAll("([^\\\\])\\.", "$1_") + .replaceAll("\\\\\\.", ".") + .replaceAll("^\\.", "_"); + } else { + return pattern + .replaceAll("([^\\\\])\\.\\*", "$1%") + .replaceAll("\\\\\\.\\*", "*") + .replaceAll("^\\.\\*", "%") + .replaceAll("([^\\\\])\\.", "$1_") + .replaceAll("\\\\\\.", ".") + .replaceAll("^\\.", "_"); + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index d444fd10a3808..4de668a9e5686 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationTyp import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetColumnsOperation +import org.apache.hive.service.cli.operation.MetadataOperationUtils._ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 19ef391a5031a..b4ce7a4392d83 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationTyp import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetFunctionsOperation import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG +import org.apache.hive.service.cli.operation.MetadataOperationUtils._ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index b5fe5001e9351..7ebe8ee928d28 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetSchemasOperation import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG +import org.apache.hive.service.cli.operation.MetadataOperationUtils._ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index f14f9a973d0de..14c308a42ee0a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -25,6 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils} import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTablesOperation +import org.apache.hive.service.cli.operation.MetadataOperationUtils._ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala new file mode 100644 index 0000000000000..ed290be0b9248 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala @@ -0,0 +1,51 @@ +/* + * 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. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hive.service.cli.operation.MetadataOperationUtils + +import org.apache.spark.SparkFunSuite + +class MetadataOperationUtilsSuite extends SparkFunSuite { + + test("convertSchemaPattern") { + Seq(("", "*"), ("%", "*"), (null, "*"), + (".*", ".*"), ("_*", ".*"), ("_%", ".*"), (".%", ".*"), + ("db%", "db*"), ("db*", "db*"), + ("db_", "db."), ("db.", "db."), + ("*", "*")) foreach { v => + val schemaPattern = MetadataOperationUtils.convertSchemaPattern(v._1) + assert(schemaPattern == v._2) + } + } + + test("newConvertSchemaPattern") { + Seq(("", "%", "%"), ("%", "%", "%"), (null, "%", "%"), + (".*", "_%", "%"), ("_*", "_%", "_*"), ("_%", "_%", "_%"), (".%", "_%", "_%"), + ("db%", "db%", "db%"), ("db*", "db%", "db*"), + ("db_", "db_", "db_"), ("db.", "db_", "db_"), + ("*", "%", "*")) foreach { v => + val schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(v._1, true) + assert(schemaPattern == v._2) + + val schemaPatternDatanucleusFormat = + MetadataOperationUtils.newConvertSchemaPattern(v._1, false) + assert(schemaPatternDatanucleusFormat == v._3) + } + } +} From 2aeb62068a98e351728584ae49033542092aa4ce Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 28 Feb 2024 11:02:27 +0800 Subject: [PATCH 10/11] improve --- docs/sql-ref-syntax-aux-show-tables.md | 17 ++++++++++------- .../sql/connector/catalog/TableCatalog.java | 4 ++-- .../sql/catalyst/catalog/SessionCatalog.scala | 6 +++--- .../spark/sql/catalyst/util/StringUtils.scala | 7 +++++-- .../spark/sql/execution/command/functions.scala | 4 ++-- .../spark/sql/execution/command/views.scala | 2 +- .../datasources/v2/ShowFunctionsExec.scala | 2 +- .../spark/sql/hive/HiveExternalCatalog.scala | 2 +- 8 files changed, 25 insertions(+), 19 deletions(-) diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md index f53d1ed145c4d..d3754b303f3b3 100644 --- a/docs/sql-ref-syntax-aux-show-tables.md +++ b/docs/sql-ref-syntax-aux-show-tables.md @@ -42,16 +42,19 @@ SHOW TABLES [ { FROM | IN } database_name ] [ LIKE regex_pattern ] Specifies the regular expression pattern that is used to filter out unwanted tables. - After Spark Version 4.0 - * We accept only SQL type like expressions containing '%' for any character(s), and '_' for a single character. - * Examples are 'employees', 'emp%', 'emplo_ees', all of which will match the database named 'employees'. - - Before Spark Version 4.0 + 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 are `'employees'`, `'emp*'`, `'emp*|*ees'`, all of which will match the database named `'employees'`. ### Examples @@ -94,7 +97,7 @@ SHOW TABLES FROM default LIKE 'sam%'; +--------+---------+-----------+ -- List all tables matching the pattern `sam*|suj` -Note: After Spark Version 4.0, the `OR` syntax represented by `|` is no longer supported by default. +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/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 1745d0e047c82..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 @@ -109,10 +109,10 @@ public interface TableCatalog extends CatalogPlugin { * * @param namespace a multi-part namespace * @param pattern the filter pattern, - * when SQLConf.LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_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:// + * 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 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 e1dbd9dec35a0..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 @@ -1034,7 +1034,7 @@ class SessionCatalog( * temporary views. */ def listTables(db: String): Seq[TableIdentifier] = - listTables(db, StringUtils.getAllMatchWildcard) + listTables(db, StringUtils.getMatchAllWildcard) /** * List all matching tables in the specified database, including local temporary views. @@ -1832,7 +1832,7 @@ class SessionCatalog( * defined). */ def listFunctions(db: String): Seq[(FunctionIdentifier, String)] = - listFunctions(db, StringUtils.getAllMatchWildcard) + listFunctions(db, StringUtils.getMatchAllWildcard) /** * List all matching functions in the specified database, including temporary functions. This @@ -1884,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, StringUtils.getAllMatchWildcard).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 6d223c91da89a..c02d9860df588 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 @@ -107,7 +107,10 @@ object StringUtils extends Logging { def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s.trimAll().toLowerCase) // scalastyle:on caselocale - def getAllMatchWildcard: String = { + /** + * get Wildcard that represent matching "all" + */ + def getMatchAllWildcard: String = { if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { "*" } else { @@ -148,7 +151,7 @@ object StringUtils extends Logging { /** * 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, SQL type like expressions: + * @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 */ 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 5ee62d77912dd..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(StringUtils.getAllMatchWildcard)) + .listFunctions(db, pattern.getOrElse(StringUtils.getMatchAllWildcard)) .collect { case (f, "USER") if showUserFunctions => f.unquotedString case (f, "SYSTEM") if showSystemFunctions => f.unquotedString @@ -187,7 +187,7 @@ case class ShowFunctionsCommand( if (showSystemFunctions) { (functionNames ++ StringUtils.filterPattern(FunctionRegistry.builtinOperators.keys.toSeq, - pattern.getOrElse(StringUtils.getAllMatchWildcard))) + 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 c84d39e81daab..a8ff438d5443a 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 @@ -336,7 +336,7 @@ case class ShowViewsCommand( // Show the information of views. val views = tableIdentifierPattern.map(catalog.listViews(databaseName, _)) - .getOrElse(catalog.listViews(databaseName, StringUtils.getAllMatchWildcard)) + .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 22870cee566e5..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.getAllMatchWildcard)) + StringUtils.filterPattern(names, pattern.getOrElse(StringUtils.getMatchAllWildcard)) } override protected def run(): Seq[InternalRow] = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 95ed73f204fba..e164135162a57 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -191,7 +191,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } override def listDatabases(): Seq[String] = withClient { - client.listDatabases(StringUtils.getAllMatchWildcard) + client.listDatabases(StringUtils.getMatchAllWildcard) } override def listDatabases(pattern: String): Seq[String] = withClient { From 7abf24163ccd830eeb28ca258468f739188a9e3d Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 28 Feb 2024 17:23:04 +0800 Subject: [PATCH 11/11] fix style --- .../spark/sql/catalyst/util/StringUtils.scala | 4 +- .../sql/catalyst/util/StringUtilsSuite.scala | 20 ++--- .../command/ShowTablesSuiteBase.scala | 4 +- .../command/v1/ShowTablesSuite.scala | 1 - .../cli/operation/GetColumnsOperation.java | 12 +-- .../cli/operation/GetFunctionsOperation.java | 4 +- .../cli/operation/GetSchemasOperation.java | 4 +- .../cli/operation/GetTablesOperation.java | 10 +-- .../cli/operation/MetadataOperationUtils.java | 34 ++++---- .../SparkGetColumnsOperation.scala | 8 +- .../SparkGetFunctionsOperation.scala | 4 +- .../SparkGetSchemasOperation.scala | 4 +- .../SparkGetTablesOperation.scala | 4 +- .../MetadataOperationUtilsSuite.scala | 10 +-- .../spark/sql/hive/StatisticsSuite.scala | 4 +- .../sql/hive/execution/HiveCommandSuite.scala | 87 +++++++++---------- 16 files changed, 104 insertions(+), 110 deletions(-) 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 c02d9860df588..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 @@ -120,7 +120,7 @@ object StringUtils extends Logging { def filterPattern(names: Seq[String], pattern: String): Seq[String] = { if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { - filterPatternLegacy(names, pattern) + legacyFilterPattern(names, pattern) } else { filterBySQLLikePattern(names, pattern) } @@ -135,7 +135,7 @@ object StringUtils extends Logging { * on both ends will be ignored * @return the filtered names list in order */ - def filterPatternLegacy(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 { 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 9a38f86f86967..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(filterPatternLegacy(names, " * ") === Seq("a1", "a2", "b2", "c3")) - assert(filterPatternLegacy(names, "*a*") === Seq("a1", "a2")) - assert(filterPatternLegacy(names, " *a* ") === Seq("a1", "a2")) - assert(filterPatternLegacy(names, " a* ") === Seq("a1", "a2")) - assert(filterPatternLegacy(names, " a.* ") === Seq("a1", "a2")) - assert(filterPatternLegacy(names, " B.*|a* ") === Seq("a1", "a2", "b2")) - assert(filterPatternLegacy(names, " a. ") === Seq("a1", "a2")) - assert(filterPatternLegacy(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") { @@ -165,7 +165,7 @@ class StringUtilsSuite extends SparkFunSuite with SQLHelper { assert(likePatternToRegExp(" d* ") === "\\Q \\E\\Qd\\E\\Q*\\E\\Q \\E") } - test("filter like pattern") { + test("filterBySQLLikePattern") { val names = Seq("a1", "a2", "b2", "c3") assert(filterBySQLLikePattern(names, "%") === Seq("a1", "a2", "b2", "c3")) assert(filterBySQLLikePattern(names, "%a%") === Seq("a1", "a2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index 76871a520e682..3f9a81730f9bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -284,8 +284,8 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE TABLE $catalog.$namespace.$table2 (data2 string, id2 bigint) " + s"$defaultUsing PARTITIONED BY (id2)") - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table%'"). - sort("tableName") + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table%'") + .sort("tableName") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) val resultCollect = result.collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index 1fbcc1265223a..bbe19b5504bdc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -66,7 +66,6 @@ trait ShowTablesSuiteBase extends command.ShowTablesSuiteBase with command.Tests val expected = Seq(Row("", "source", true), Row("", "source2", true)) val df = sql("SHOW TABLE EXTENDED FROM default LIKE '%source%'") - val result = df.collect() val resultWithoutInfo = result.map { case Row(db, table, temp, _) => Row(db, table, temp) } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java index acf579ca3f5fa..ece46aa17ec77 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -139,18 +139,18 @@ public void runInternal() throws HiveSQLException { String schemaPattern, tablePattern; Pattern columnPattern = null; if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { - schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); - tablePattern = MetadataOperationUtils.convertIdentifierPattern(tableName, true); + schemaPattern = MetadataOperationUtils.legacyConvertSchemaPattern(schemaName); + tablePattern = MetadataOperationUtils.legacyConvertIdentifierPattern(tableName, true); if (columnName != null) { columnPattern = Pattern.compile( - MetadataOperationUtils.convertIdentifierPattern(columnName, false)); + MetadataOperationUtils.legacyConvertIdentifierPattern(columnName, false)); } } else { - schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); - tablePattern = MetadataOperationUtils.newConvertIdentifierPattern(tableName, true); + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName, true); + tablePattern = MetadataOperationUtils.convertIdentifierPattern(tableName, true); if (columnName != null) { columnPattern = Pattern.compile( - MetadataOperationUtils.newConvertIdentifierPattern(columnName, false)); + MetadataOperationUtils.convertIdentifierPattern(columnName, false)); } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java index 7611d0b85834d..b0559c93dc232 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -85,9 +85,9 @@ public void runInternal() throws HiveSQLException { IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); String schemaPattern; if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { - schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); + schemaPattern = MetadataOperationUtils.legacyConvertSchemaPattern(schemaName); } else { - schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName, true); } List matchingDbs; try { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index 9962236ddf2b0..66ec7a0dd53bb 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -65,9 +65,9 @@ public void runInternal() throws HiveSQLException { IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); String schemaPattern; if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { - schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); + schemaPattern = MetadataOperationUtils.legacyConvertSchemaPattern(schemaName); } else { - schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName, true); } for (String dbName : metastoreClient.getDatabases(schemaPattern)) { rowSet.addRow(new Object[] {dbName, DEFAULT_HIVE_CATALOG}); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java index b8bd50687aa17..86595e20fc034 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -95,16 +95,14 @@ public void runInternal() throws HiveSQLException { setState(OperationState.RUNNING); try { IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern, tablePattern; if (SQLConf.get().legacyUseStarAndVerticalBarAsWildcardsInLikePattern()) { - schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName); - tablePattern = MetadataOperationUtils.convertIdentifierPattern(tableName, true); + schemaPattern = MetadataOperationUtils.legacyConvertSchemaPattern(schemaName); + tablePattern = MetadataOperationUtils.legacyConvertIdentifierPattern(tableName, true); } else { - schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(schemaName, true); - tablePattern = MetadataOperationUtils.newConvertIdentifierPattern(tableName, true); + schemaPattern = MetadataOperationUtils.convertSchemaPattern(schemaName, true); + tablePattern = MetadataOperationUtils.convertIdentifierPattern(tableName, true); } - List matchingDbs = metastoreClient.getDatabases(schemaPattern); if(isAuthV2Enabled()){ List privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java index 903c6e1f18cba..788f490ecbd99 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperationUtils.java @@ -22,11 +22,12 @@ public class MetadataOperationUtils { /** * Convert wildchars and escape sequence from JDBC format to datanucleous/regex */ - public static String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { + public static String legacyConvertIdentifierPattern( + final String pattern, boolean datanucleusFormat) { if (pattern == null) { - return convertPattern("%", true); + return legacyConvertPattern("%", true); } else { - return convertPattern(pattern, datanucleusFormat); + return legacyConvertPattern(pattern, datanucleusFormat); } } @@ -34,11 +35,11 @@ public static String convertIdentifierPattern(final String pattern, boolean data * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex * The schema pattern treats empty string also as wildchar */ - public static String convertSchemaPattern(final String pattern) { + public static String legacyConvertSchemaPattern(final String pattern) { if ((pattern == null) || pattern.isEmpty()) { - return convertPattern("%", true); + return legacyConvertPattern("%", true); } else { - return convertPattern(pattern, true); + return legacyConvertPattern(pattern, true); } } @@ -47,15 +48,14 @@ public static String convertSchemaPattern(final String pattern) { * Java regex patterns. * * @param pattern input which may contain '%' or '_' wildcard characters, or - * these characters escaped using {@link #getSearchStringEscape()}. - * @return replace %/_ with regex search characters, also handle escaped - * characters. + * these characters escaped using "\". + * @return replace %/_ with regex search characters, also handle escaped characters. * * The datanucleus module expects the wildchar as '*'. The columns search on the * other hand is done locally inside the hive code and that requires the regex wildchar * format '.*' This is driven by the datanucleusFormat flag. */ - public static String convertPattern(final String pattern, boolean datanucleusFormat) { + public static String legacyConvertPattern(final String pattern, boolean datanucleusFormat) { String wStr; if (datanucleusFormat) { wStr = "*"; @@ -67,25 +67,25 @@ public static String convertPattern(final String pattern, boolean datanucleusFor .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); } - public static String newConvertIdentifierPattern( + public static String convertIdentifierPattern( final String pattern, boolean datanucleusFormat) { if (pattern == null) { - return newConvertPattern("%", true); + return convertPattern("%", true); } else { - return newConvertPattern(pattern, datanucleusFormat); + return convertPattern(pattern, datanucleusFormat); } } - public static String newConvertSchemaPattern(final String pattern, boolean datanucleusFormat) { + public static String convertSchemaPattern(final String pattern, boolean datanucleusFormat) { if ((pattern == null) || pattern.isEmpty()) { String all = datanucleusFormat? "*" : ".*"; - return newConvertPattern(all, datanucleusFormat); + return convertPattern(all, datanucleusFormat); } else { - return newConvertPattern(pattern, datanucleusFormat); + return convertPattern(pattern, datanucleusFormat); } } - public static String newConvertPattern(final String pattern, boolean datanucleusFormat) { + public static String convertPattern(final String pattern, boolean datanucleusFormat) { if (datanucleusFormat) { return pattern .replaceAll("([^\\\\])\\*", "$1%") diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 4de668a9e5686..27e524415d11f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -78,17 +78,17 @@ private[hive] class SparkGetColumnsOperation( val (schemaPattern, tablePattern) = if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { - (convertSchemaPattern(schemaName), convertIdentifierPattern(tableName, true)) + (legacyConvertSchemaPattern(schemaName), legacyConvertIdentifierPattern(tableName, true)) } else { - (newConvertSchemaPattern(schemaName, true), newConvertIdentifierPattern(tableName, true)) + (convertSchemaPattern(schemaName, true), convertIdentifierPattern(tableName, true)) } var columnPattern: Pattern = null if (columnName != null) { if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { - columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)) + columnPattern = Pattern.compile(legacyConvertIdentifierPattern(columnName, false)) } else { - columnPattern = Pattern.compile(newConvertIdentifierPattern(columnName, false)) + columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index b4ce7a4392d83..649c81750652f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -65,9 +65,9 @@ private[hive] class SparkGetFunctionsOperation( // get databases for schema pattern val (schemaPattern, functionPattern) = if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { - (convertSchemaPattern(schemaName), convertSchemaPattern(functionName)) + (legacyConvertSchemaPattern(schemaName), legacyConvertSchemaPattern(functionName)) } else { - (newConvertSchemaPattern(schemaName, true), newConvertSchemaPattern(functionName, true)) + (convertSchemaPattern(schemaName, true), convertSchemaPattern(functionName, true)) } val matchingDbs = catalog.listDatabases(schemaPattern) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 7ebe8ee928d28..63eeb7da3f7a2 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -71,9 +71,9 @@ private[hive] class SparkGetSchemasOperation( try { val schemaPattern = if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { - convertSchemaPattern(schemaName) + legacyConvertSchemaPattern(schemaName) } else { - newConvertSchemaPattern(schemaName, true) + convertSchemaPattern(schemaName, true) } sqlContext.sessionState.catalog.listDatabases(schemaPattern).foreach { dbName => rowSet.addRow(Array[AnyRef](dbName, DEFAULT_HIVE_CATALOG)) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 14c308a42ee0a..14695d49ce42a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -68,9 +68,9 @@ private[hive] class SparkGetTablesOperation( val catalog = sqlContext.sessionState.catalog val (schemaPattern, tablePattern) = if (SQLConf.get.legacyUseStarAndVerticalBarAsWildcardsInLikePattern) { - (convertSchemaPattern(schemaName), convertIdentifierPattern(tableName, true)) + (legacyConvertSchemaPattern(schemaName), legacyConvertIdentifierPattern(tableName, true)) } else { - (newConvertSchemaPattern(schemaName, true), newConvertIdentifierPattern(tableName, true)) + (convertSchemaPattern(schemaName, true), convertIdentifierPattern(tableName, true)) } val matchingDbs = catalog.listDatabases(schemaPattern) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala index ed290be0b9248..1629acf742d34 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/MetadataOperationUtilsSuite.scala @@ -23,28 +23,28 @@ import org.apache.spark.SparkFunSuite class MetadataOperationUtilsSuite extends SparkFunSuite { - test("convertSchemaPattern") { + test("legacy convertSchemaPattern") { Seq(("", "*"), ("%", "*"), (null, "*"), (".*", ".*"), ("_*", ".*"), ("_%", ".*"), (".%", ".*"), ("db%", "db*"), ("db*", "db*"), ("db_", "db."), ("db.", "db."), ("*", "*")) foreach { v => - val schemaPattern = MetadataOperationUtils.convertSchemaPattern(v._1) + val schemaPattern = MetadataOperationUtils.legacyConvertSchemaPattern(v._1) assert(schemaPattern == v._2) } } - test("newConvertSchemaPattern") { + test("new convertSchemaPattern") { Seq(("", "%", "%"), ("%", "%", "%"), (null, "%", "%"), (".*", "_%", "%"), ("_*", "_%", "_*"), ("_%", "_%", "_%"), (".%", "_%", "_%"), ("db%", "db%", "db%"), ("db*", "db%", "db*"), ("db_", "db_", "db_"), ("db.", "db_", "db_"), ("*", "%", "*")) foreach { v => - val schemaPattern = MetadataOperationUtils.newConvertSchemaPattern(v._1, true) + val schemaPattern = MetadataOperationUtils.convertSchemaPattern(v._1, true) assert(schemaPattern == v._2) val schemaPatternDatanucleusFormat = - MetadataOperationUtils.newConvertSchemaPattern(v._1, false) + MetadataOperationUtils.convertSchemaPattern(v._1, false) assert(schemaPatternDatanucleusFormat == v._3) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 006daff3ac721..7ee6d53fb126b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -870,9 +870,9 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_))) if (analyzedByHive) { - assert(StringUtils.filterPatternLegacy(describeResult1, "*numRows\\s+500*").nonEmpty) + assert(StringUtils.legacyFilterPattern(describeResult1, "*numRows\\s+500*").nonEmpty) } else { - assert(StringUtils.filterPatternLegacy(describeResult1, "*numRows\\s+500*").isEmpty) + assert(StringUtils.legacyFilterPattern(describeResult1, "*numRows\\s+500*").isEmpty) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 58fd4d29e3be7..a38aeebd13557 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -78,52 +78,49 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("show views") { - withSQLConf( - SQLConf.LEGACY_USE_STAR_AND_VERTICAL_BAR_AS_WILDCARDS_IN_LIKE_PATTERN.key -> "true") { - withView("show1a", "show2b", "global_temp.temp1", "temp2") { - sql("CREATE VIEW show1a AS SELECT 1 AS id") - sql("CREATE VIEW show2b AS SELECT 1 AS id") - sql("CREATE GLOBAL TEMP VIEW temp1 AS SELECT 1 AS id") - sql("CREATE TEMP VIEW temp2 AS SELECT 1 AS id") - checkAnswer( - sql("SHOW VIEWS"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: - Row("default", "parquet_view1", false) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS IN default"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: - Row("default", "parquet_view1", false) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS FROM default"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: - Row("default", "parquet_view1", false) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS FROM global_temp"), - Row("global_temp", "temp1", true) :: - Row("", "temp2", true) :: Nil) - checkAnswer( - sql("SHOW VIEWS 'show1*|show2*'"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: Nil) - checkAnswer( - sql("SHOW VIEWS LIKE 'show1*|show2*'"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: Nil) - checkAnswer( - sql("SHOW VIEWS IN default 'show1*'"), - Row("default", "show1a", false) :: Nil) - checkAnswer( - sql("SHOW VIEWS IN default LIKE 'show1*|show2*'"), - Row("default", "show1a", false) :: - Row("default", "show2b", false) :: Nil) + withView("show1a", "show2b", "global_temp.temp1", "temp2") { + sql("CREATE VIEW show1a AS SELECT 1 AS id") + sql("CREATE VIEW show2b AS SELECT 1 AS id") + sql("CREATE GLOBAL TEMP VIEW temp1 AS SELECT 1 AS id") + sql("CREATE TEMP VIEW temp2 AS SELECT 1 AS id") + checkAnswer( + sql("SHOW VIEWS"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS FROM default"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS FROM global_temp"), + Row("global_temp", "temp1", true) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS 'show%'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS LIKE 'show%'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default 'show1%'"), + Row("default", "show1a", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default LIKE 'show%'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) } - } } Seq(true, false).foreach { local =>