diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 4f89b462a6ce3..0f89242b2ecb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -222,6 +222,33 @@ case class UnresolvedStar(target: Option[Seq[String]]) extends Star with Unevalu override def toString: String = target.map(_ + ".").getOrElse("") + "*" } +/** + * Represents all of the input attributes to a given relational operator, for example in + * "SELECT `(id)?+.+` FROM ...". + * + * @param table an optional table that should be the target of the expansion. If omitted all + * tables' columns are produced. + */ +case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable { + + override def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] = { + val expandedAttributes: Seq[Attribute] = table match { + // If there is no table specified, use all input attributes that match expr + case None => input.filter(_.name.matches(expr)) + // If there is a table, pick out attributes that are part of this table that match expr + case Some(t) => input.filter(_.qualifiers.filter(resolver(_, t)).nonEmpty) + .filter(_.name.matches(expr)) + } + expandedAttributes.zip(input).map { + case (n: NamedExpression, _) => n + case (e, originalAttribute) => + Alias(e, originalAttribute.name)(qualifiers = originalAttribute.qualifiers) + } + } + + override def toString: String = table.map(_ + ".").getOrElse("") + expr +} + /** * Used to assign new names to Generator's output, such as hive udtf. * For example the SQL expression "stack(2, key, value, key, value) as (a, b)" could be represented diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 091caab921fe9..e206b01957f64 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1493,7 +1493,11 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C /* Attribute References */ case Token("TOK_TABLE_OR_COL", Token(name, Nil) :: Nil) => - UnresolvedAttribute.quoted(cleanIdentifier(name)) + name match { + // If the columns is wrapped in backticks, treat it as a regular expression + case escapedIdentifier(i) => UnresolvedRegex(i, None) + case _ => UnresolvedAttribute.quoted(name) + } case Token(".", qualifier :: Token(attr, Nil) :: Nil) => nodeToExpr(qualifier) match { case UnresolvedAttribute(nameParts) =>