aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorbomeng <bmeng@us.ibm.com>2016-04-06 11:05:52 -0700
committerAndrew Or <andrew@databricks.com>2016-04-06 11:06:14 -0700
commit5abd02c02b3fa3505defdc8ab0c5c5e23a16aa80 (patch)
tree88ffa4ba214811f3242254bd280bed48bf463ede
parent10494feae0c2c1aca545c73ba61af6d8f743c5bb (diff)
downloadspark-5abd02c02b3fa3505defdc8ab0c5c5e23a16aa80.tar.gz
spark-5abd02c02b3fa3505defdc8ab0c5c5e23a16aa80.tar.bz2
spark-5abd02c02b3fa3505defdc8ab0c5c5e23a16aa80.zip
[SPARK-14429][SQL] Improve LIKE pattern in "SHOW TABLES / FUNCTIONS LIKE <pattern>" DDL
LIKE <pattern> is commonly used in SHOW TABLES / FUNCTIONS etc DDL. In the pattern, user can use `|` or `*` as wildcards. 1. Currently, we used `replaceAll()` to replace `*` with `.*`, but the replacement was scattered in several places; I have created an utility method and use it in all the places; 2. Consistency with Hive: the pattern is case insensitive in Hive and white spaces will be trimmed, but current pattern matching does not do that. For example, suppose we have tables (t1, t2, t3), `SHOW TABLES LIKE ' T* ' ` will list all the t-tables. Please use Hive to verify it. 3. Combined with `|`, the result will be sorted. For pattern like `' B*|a* '`, it will list the result in a-b order. I've made some changes to the utility method to make sure we will get the same result as Hive does. A new method was created in StringUtil and test cases were added. andrewor14 Author: bomeng <bmeng@us.ibm.com> Closes #12206 from bomeng/SPARK-14429.
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala13
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala10
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala23
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala12
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala12
5 files changed, 46 insertions, 24 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
index 2af0107fa3..5d136b663f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
@@ -21,7 +21,7 @@ import scala.collection.mutable
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
-
+import org.apache.spark.sql.catalyst.util.StringUtils
/**
* An in-memory (ephemeral) implementation of the system catalog.
@@ -47,11 +47,6 @@ class InMemoryCatalog extends ExternalCatalog {
// Database name -> description
private val catalog = new scala.collection.mutable.HashMap[String, DatabaseDesc]
- private def filterPattern(names: Seq[String], pattern: String): Seq[String] = {
- val regex = pattern.replaceAll("\\*", ".*").r
- names.filter { funcName => regex.pattern.matcher(funcName).matches() }
- }
-
private def functionExists(db: String, funcName: String): Boolean = {
requireDbExists(db)
catalog(db).functions.contains(funcName)
@@ -141,7 +136,7 @@ class InMemoryCatalog extends ExternalCatalog {
}
override def listDatabases(pattern: String): Seq[String] = synchronized {
- filterPattern(listDatabases(), pattern)
+ StringUtils.filterPattern(listDatabases(), pattern)
}
override def setCurrentDatabase(db: String): Unit = { /* no-op */ }
@@ -208,7 +203,7 @@ class InMemoryCatalog extends ExternalCatalog {
}
override def listTables(db: String, pattern: String): Seq[String] = synchronized {
- filterPattern(listTables(db), pattern)
+ StringUtils.filterPattern(listTables(db), pattern)
}
// --------------------------------------------------------------------------
@@ -322,7 +317,7 @@ class InMemoryCatalog extends ExternalCatalog {
override def listFunctions(db: String, pattern: String): Seq[String] = synchronized {
requireDbExists(db)
- filterPattern(catalog(db).functions.keysIterator.toSeq, pattern)
+ StringUtils.filterPattern(catalog(db).functions.keysIterator.toSeq, pattern)
}
}
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 62a3b1c105..2acf584e8f 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
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchFunctionE
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
-
+import org.apache.spark.sql.catalyst.util.StringUtils
/**
* An internal catalog that is used by a Spark Session. This internal catalog serves as a
@@ -297,9 +297,7 @@ class SessionCatalog(
def listTables(db: String, pattern: String): Seq[TableIdentifier] = {
val dbTables =
externalCatalog.listTables(db, pattern).map { t => TableIdentifier(t, Some(db)) }
- val regex = pattern.replaceAll("\\*", ".*").r
- val _tempTables = tempTables.keys.toSeq
- .filter { t => regex.pattern.matcher(t).matches() }
+ val _tempTables = StringUtils.filterPattern(tempTables.keys.toSeq, pattern)
.map { t => TableIdentifier(t) }
dbTables ++ _tempTables
}
@@ -613,9 +611,7 @@ class SessionCatalog(
def listFunctions(db: String, pattern: String): Seq[FunctionIdentifier] = {
val dbFunctions =
externalCatalog.listFunctions(db, pattern).map { f => FunctionIdentifier(f, Some(db)) }
- val regex = pattern.replaceAll("\\*", ".*").r
- val loadedFunctions = functionRegistry.listFunction()
- .filter { f => regex.pattern.matcher(f).matches() }
+ val loadedFunctions = StringUtils.filterPattern(functionRegistry.listFunction(), pattern)
.map { f => FunctionIdentifier(f) }
// TODO: Actually, there will be dbFunctions that have been loaded into the FunctionRegistry.
// So, the returned list may have two entries for the same function.
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 c2eeb3c565..0f65028261 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
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.util
-import java.util.regex.Pattern
+import java.util.regex.{Pattern, PatternSyntaxException}
import org.apache.spark.unsafe.types.UTF8String
@@ -52,4 +52,25 @@ object StringUtils {
def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.toLowerCase)
def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s.toLowerCase)
+
+ /**
+ * 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, only '*' and '|' are allowed as wildcards, others will
+ * follows regular expression convention, case insensitive match and white spaces
+ * on both ends will be ignored
+ * @return the filtered names list in order
+ */
+ def filterPattern(names: Seq[String], pattern: String): Seq[String] = {
+ 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() }
+ } catch {
+ case _: PatternSyntaxException =>
+ }
+ }
+ funcNames.toSeq
+ }
}
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 d6f273f9e5..2ffc18a8d1 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
@@ -31,4 +31,16 @@ class StringUtilsSuite extends SparkFunSuite {
assert(escapeLikeRegex("**") === "(?s)\\Q*\\E\\Q*\\E")
assert(escapeLikeRegex("a_b") === "(?s)\\Qa\\E.\\Qb\\E")
}
+
+ 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)
+ }
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 5a851b47ca..2ab7c1581c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -24,6 +24,7 @@ import org.apache.spark.AccumulatorSuite
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.expressions.SortOrder
import org.apache.spark.sql.catalyst.plans.logical.Aggregate
+import org.apache.spark.sql.catalyst.util.StringUtils
import org.apache.spark.sql.execution.aggregate
import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, CartesianProduct, SortMergeJoin}
import org.apache.spark.sql.functions._
@@ -56,17 +57,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
test("show functions") {
def getFunctions(pattern: String): Seq[Row] = {
- val regex = java.util.regex.Pattern.compile(pattern)
- sqlContext.sessionState.functionRegistry.listFunction()
- .filter(regex.matcher(_).matches()).map(Row(_))
+ StringUtils.filterPattern(sqlContext.sessionState.functionRegistry.listFunction(), pattern)
+ .map(Row(_))
}
- checkAnswer(sql("SHOW functions"), getFunctions(".*"))
+ checkAnswer(sql("SHOW functions"), getFunctions("*"))
Seq("^c*", "*e$", "log*", "*date*").foreach { pattern =>
// For the pattern part, only '*' and '|' are allowed as wildcards.
// For '*', we need to replace it to '.*'.
- checkAnswer(
- sql(s"SHOW FUNCTIONS '$pattern'"),
- getFunctions(pattern.replaceAll("\\*", ".*")))
+ checkAnswer(sql(s"SHOW FUNCTIONS '$pattern'"), getFunctions(pattern))
}
}